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-cli commons-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.flink flink-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} provided org.apache.flink - flink-java + flink-streaming-java + ${project.version} + provided + + + + org.apache.flink + flink-streaming-java ${project.version} test-jar test + + org.apache.flink + flink-connector-files + ${project.version} + test + + org.apache.hadoop hadoop-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.flink flink-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.flink flink-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.flink flink-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.flink flink-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.flink flink-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.flink flink-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-test flink-parent-child-classloading-test-program flink-parent-child-classloading-test-lib-package - flink-dataset-allround-test - flink-dataset-fine-grained-recovery-test flink-datastream-allround-test flink-batch-sql-test flink-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 - *
    - *
  1. Int: Number of data points - *
  2. Int: Number of cluster centers - *
  3. Optional String: Output path, default value is {tmp.dir} - *
  4. Optional Double: Standard deviation of data points - *
  5. Optional Double: Value range of cluster centers - *
  6. 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 out) - throws Exception { - LOGGER.info("Processing task: " + task); - Path outPath = new Path(targetPath, task.getRelativePath()); - - FileSystem targetFs = targetPath.getFileSystem(); - // creating parent folders in case of a local FS - if (!targetFs.isDistributedFS()) { - // dealing with cases like file:///tmp or just /tmp - File outFile = - outPath.toUri().isAbsolute() - ? new File(outPath.toUri()) - : new File(outPath.toString()); - File parentFile = outFile.getParentFile(); - if (!parentFile.mkdirs() && !parentFile.exists()) { - throw new RuntimeException( - "Cannot create local file system directories: " - + parentFile); - } - } - FSDataOutputStream outputStream = null; - FSDataInputStream inputStream = null; - try { - outputStream = - targetFs.create( - outPath, FileSystem.WriteMode.OVERWRITE); - inputStream = - task.getPath().getFileSystem().open(task.getPath()); - int bytes = IOUtils.copy(inputStream, outputStream); - bytesCounter.add(bytes); - } finally { - IOUtils.closeQuietly(inputStream); - IOUtils.closeQuietly(outputStream); - } - fileCounter.add(1L); - } - }); - - // no data sinks are needed, therefore just printing an empty result - res.print(); - - Map accumulators = - env.getLastJobExecutionResult().getAllAccumulatorResults(); - LOGGER.info("== COUNTERS =="); - for (Map.Entry e : accumulators.entrySet()) { - LOGGER.info(e.getKey() + ": " + e.getValue()); - } - } - - // ----------------------------------------------------------------------------------------- - // HELPER METHODS - // ----------------------------------------------------------------------------------------- - - private static boolean isLocal(final ExecutionEnvironment env) { - return env instanceof LocalEnvironment; - } - - private static boolean isOnDistributedFS(final Path path) throws IOException { - return path.getFileSystem().isDistributedFS(); - } - - private static List getCopyTasks(Path sourcePath) throws IOException { - List tasks = new ArrayList<>(); - getCopyTasks(sourcePath, "", tasks); - return tasks; - } - - private static void getCopyTasks(Path p, String rel, List tasks) - throws IOException { - FileStatus[] res = p.getFileSystem().listStatus(p); - if (res == null) { - return; - } - for (FileStatus fs : res) { - if (fs.isDir()) { - getCopyTasks(fs.getPath(), rel + fs.getPath().getName() + "/", tasks); - } else { - Path cp = fs.getPath(); - tasks.add(new FileCopyTask(cp, rel + cp.getName())); - } - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTask.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTask.java deleted file mode 100644 index 7a1353251f952..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTask.java +++ /dev/null @@ -1,55 +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.core.fs.Path; - -import org.apache.commons.lang3.StringUtils; - -import java.io.Serializable; - -/** A Java POJO that represents a task for copying a single file. */ -public class FileCopyTask implements Serializable { - - private static final long serialVersionUID = -8760082278978316032L; - - private final Path path; - private final String relativePath; - - public FileCopyTask(Path path, String relativePath) { - if (StringUtils.isEmpty(relativePath)) { - throw new IllegalArgumentException("Relative path should not be empty for: " + path); - } - this.path = path; - this.relativePath = relativePath; - } - - public Path getPath() { - return path; - } - - public String getRelativePath() { - return relativePath; - } - - @Override - public String toString() { - return "FileCopyTask{" + "path=" + path + ", relativePath='" + relativePath + '\'' + '}'; - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTaskInputFormat.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTaskInputFormat.java deleted file mode 100644 index b7f8b6c3ceaaa..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTaskInputFormat.java +++ /dev/null @@ -1,125 +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.io.InputFormat; -import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.util.Preconditions; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; - -/** - * An implementation of an input format that dynamically assigns {@code FileCopyTask} to the mappers - * that have finished previously assigned tasks. - */ -public class FileCopyTaskInputFormat implements InputFormat { - - private static final long serialVersionUID = -644394866425221151L; - - private static final Logger LOGGER = LoggerFactory.getLogger(FileCopyTaskInputFormat.class); - - private final List tasks; - - public FileCopyTaskInputFormat(List tasks) { - this.tasks = tasks; - } - - private class FileCopyTaskAssigner implements InputSplitAssigner { - private Queue splits; - - public FileCopyTaskAssigner(FileCopyTaskInputSplit[] inputSplits) { - splits = new LinkedList<>(Arrays.asList(inputSplits)); - } - - @Override - public InputSplit getNextInputSplit(String host, int taskId) { - LOGGER.info("Getting copy task for task: " + taskId); - return splits.poll(); - } - - @Override - public void returnInputSplit(List splits, int taskId) { - synchronized (this.splits) { - for (InputSplit split : splits) { - Preconditions.checkState(this.splits.add((FileCopyTaskInputSplit) split)); - } - } - } - } - - @Override - public void configure(Configuration parameters) { - // no op - } - - @Override - public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { - return null; - } - - @Override - public FileCopyTaskInputSplit[] createInputSplits(int minNumSplits) throws IOException { - FileCopyTaskInputSplit[] splits = new FileCopyTaskInputSplit[tasks.size()]; - int i = 0; - for (FileCopyTask t : tasks) { - splits[i] = new FileCopyTaskInputSplit(t, i); - i++; - } - return splits; - } - - @Override - public InputSplitAssigner getInputSplitAssigner(FileCopyTaskInputSplit[] inputSplits) { - return new FileCopyTaskAssigner(inputSplits); - } - - private FileCopyTaskInputSplit curInputSplit = null; - - @Override - public void open(FileCopyTaskInputSplit split) throws IOException { - curInputSplit = split; - } - - @Override - public boolean reachedEnd() throws IOException { - return curInputSplit == null; - } - - @Override - public FileCopyTask nextRecord(FileCopyTask reuse) throws IOException { - FileCopyTask toReturn = curInputSplit.getTask(); - curInputSplit = null; - return toReturn; - } - - @Override - public void close() throws IOException { - // no op - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTaskInputSplit.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTaskInputSplit.java deleted file mode 100644 index 5fcfac2391f0f..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/FileCopyTaskInputSplit.java +++ /dev/null @@ -1,44 +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.core.io.InputSplit; - -/** Implementation of {@code InputSplit} for copying files. */ -public class FileCopyTaskInputSplit implements InputSplit { - - private static final long serialVersionUID = -7621656017747660450L; - - private final FileCopyTask task; - private final int splitNumber; - - public FileCopyTaskInputSplit(FileCopyTask task, int splitNumber) { - this.task = task; - this.splitNumber = splitNumber; - } - - public FileCopyTask getTask() { - return task; - } - - @Override - public int getSplitNumber() { - return splitNumber; - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java deleted file mode 100644 index 7fcb8fbf24925..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/ConnectedComponents.java +++ /dev/null @@ -1,255 +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.graph; - -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.JoinFunction; -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.aggregation.Aggregations; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.examples.java.graph.util.ConnectedComponentsData; -import org.apache.flink.util.Collector; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * An implementation of the connected components algorithm, using a delta iteration. - * - *

Initially, the algorithm assigns each vertex an unique ID. In each step, a vertex picks the - * minimum of its own ID and its neighbors' IDs, as its new ID and tells its neighbors about its new - * ID. After the algorithm has completed, all vertices in the same component will have the same ID. - * - *

A vertex whose component ID did not change needs not propagate its information in the next - * step. Because of that, the algorithm is easily expressible via a delta iteration. We here model - * the solution set as the vertices with their current component ids, and the workset as the changed - * vertices. Because we see all vertices initially as changed, the initial workset and the initial - * solution set are identical. Also, the delta to the solution set is consequently also the next - * workset.
- * - *

Input files are plain text files and must be formatted as follows: - * - *

    - *
  • Vertices represented as IDs and separated by new-line characters.
    - * For example "1\n2\n12\n42\n63" gives five vertices (1), (2), (12), (42), and - * (63). - *
  • Edges are represented as pairs for vertex IDs which are separated by space characters. - * Edges are separated by new-line characters.
    - * For example "1 2\n2 12\n1 12\n42 63" gives four (undirected) edges (1)-(2), - * (2)-(12), (1)-(12), and (42)-(63). - *
- * - *

Usage: - * ConnectedComponents --vertices <path> --edges <path> --output <path> --iterations <n> - *
- * If no parameters are provided, the program is run with default data from {@link - * org.apache.flink.examples.java.graph.util.ConnectedComponentsData} and 10 iterations. - * - *

This example shows how to use: - * - *

    - *
  • Delta Iterations - *
  • Generic-typed Functions - *
- * - *

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 ConnectedComponents { - - private static final Logger LOGGER = LoggerFactory.getLogger(ConnectedComponents.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - 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(); - - final int maxIterations = params.getInt("iterations", 10); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // read vertex and edge data - DataSet vertices = getVertexDataSet(env, params); - DataSet> edges = getEdgeDataSet(env, params).flatMap(new UndirectEdge()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0); - - // apply the step logic: join with the edges, select the minimum neighbor, update if the - // component of the candidate is smaller - DataSet> changes = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new ComponentIdFilter()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(changes, changes); - - // emit result - if (params.has("output")) { - result.writeAsCsv(params.get("output"), "\n", " "); - // execute program - env.execute("Connected Components Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - result.print(); - } - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** Function that turns a value into a 2-tuple where both fields are that value. */ - @ForwardedFields("*->f0") - public static final class DuplicateValue implements MapFunction> { - - @Override - public Tuple2 map(T vertex) { - return new Tuple2(vertex, vertex); - } - } - - /** - * Undirected edges by emitting for each input edge the input edges itself and an inverted - * version. - */ - public static final class UndirectEdge - implements FlatMapFunction, Tuple2> { - Tuple2 invertedEdge = new Tuple2(); - - @Override - public void flatMap(Tuple2 edge, Collector> out) { - invertedEdge.f0 = edge.f1; - invertedEdge.f1 = edge.f0; - out.collect(edge); - out.collect(invertedEdge); - } - } - - /** - * UDF that joins a (Vertex-ID, Component-ID) pair that represents the current component that a - * vertex is associated with, with a (Source-Vertex-ID, Target-VertexID) edge. The function - * produces a (Target-vertex-ID, Component-ID) pair. - */ - @ForwardedFieldsFirst("f1->f1") - @ForwardedFieldsSecond("f1->f0") - public static final class NeighborWithComponentIDJoin - implements JoinFunction, Tuple2, Tuple2> { - - @Override - public Tuple2 join( - Tuple2 vertexWithComponent, Tuple2 edge) { - return new Tuple2(edge.f1, vertexWithComponent.f1); - } - } - - /** - * Emit the candidate (Vertex-ID, Component-ID) pair if and only if the candidate component ID - * is less than the vertex's current component ID. - */ - @ForwardedFieldsFirst("*") - public static final class ComponentIdFilter - implements FlatJoinFunction< - Tuple2, Tuple2, Tuple2> { - - @Override - public void join( - Tuple2 candidate, - Tuple2 old, - Collector> out) { - if (candidate.f1 < old.f1) { - out.collect(candidate); - } - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static DataSet getVertexDataSet(ExecutionEnvironment env, ParameterTool params) { - if (params.has("vertices")) { - return env.readCsvFile(params.get("vertices")) - .types(Long.class) - .map( - new MapFunction, Long>() { - public Long map(Tuple1 value) { - return value.f0; - } - }); - } else { - System.out.println( - "Executing Connected Components example with default vertices data set."); - System.out.println("Use --vertices to specify file input."); - return ConnectedComponentsData.getDefaultVertexDataSet(env); - } - } - - private static DataSet> getEdgeDataSet( - ExecutionEnvironment env, ParameterTool params) { - if (params.has("edges")) { - return env.readCsvFile(params.get("edges")) - .fieldDelimiter(" ") - .types(Long.class, Long.class); - } else { - System.out.println( - "Executing Connected Components example with default edges data set."); - System.out.println("Use --edges to specify file input."); - return ConnectedComponentsData.getDefaultEdgeDataSet(env); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/EnumTriangles.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/EnumTriangles.java deleted file mode 100644 index 958b65a3a8329..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/EnumTriangles.java +++ /dev/null @@ -1,225 +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.graph; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.Order; -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.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.examples.java.graph.util.EnumTrianglesData; -import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Edge; -import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Triad; -import org.apache.flink.util.Collector; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * Triangle enumeration is a pre-processing step to find closely connected parts in graphs. A - * triangle consists of three edges that connect three vertices with each other. - * - *

The algorithm works as follows: It groups all edges that share a common vertex and builds - * triads, i.e., triples of vertices that are connected by two edges. Finally, all triads are - * filtered for which no third edge exists that closes the triangle. - * - *

Input files are plain text files and must be formatted as follows: - * - *

    - *
  • Edges are represented as pairs for vertex IDs which are separated by space characters. - * Edges are separated by new-line characters.
    - * For example "1 2\n2 12\n1 12\n42 63" gives four (undirected) edges (1)-(2), - * (2)-(12), (1)-(12), and (42)-(63) that include a triangle - *
- * - *
- *     (1)
- *     /  \
- *   (2)-(12)
- * 
- * - *

Usage: EnumTriangleBasic --edges <path> --output <path>
- * If no parameters are provided, the program is run with default data from {@link - * EnumTrianglesData}. - * - *

This example shows how to use: - * - *

    - *
  • Custom Java objects which extend Tuple - *
  • Group Sorting - *
- * - *

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 EnumTriangles { - - private static final Logger LOGGER = LoggerFactory.getLogger(EnumTriangles.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - 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 - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // read input data - DataSet edges; - if (params.has("edges")) { - edges = - env.readCsvFile(params.get("edges")) - .fieldDelimiter(" ") - .includeFields(true, true) - .types(Integer.class, Integer.class) - .map(new TupleEdgeConverter()); - } else { - System.out.println("Executing EnumTriangles example with default edges data set."); - System.out.println("Use --edges to specify file input."); - edges = EnumTrianglesData.getDefaultEdgeDataSet(env); - } - - // project edges by vertex id - DataSet edgesById = edges.map(new EdgeByIdProjector()); - - DataSet triangles = - edgesById - // build triads - .groupBy(Edge.V1) - .sortGroup(Edge.V2, Order.ASCENDING) - .reduceGroup(new TriadBuilder()) - // filter triads - .join(edgesById) - .where(Triad.V2, Triad.V3) - .equalTo(Edge.V1, Edge.V2) - .with(new TriadFilter()); - - // emit result - if (params.has("output")) { - triangles.writeAsCsv(params.get("output"), "\n", ","); - // execute program - env.execute("Basic Triangle Enumeration Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - triangles.print(); - } - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** Converts a Tuple2 into an Edge. */ - @ForwardedFields("0;1") - public static class TupleEdgeConverter implements MapFunction, Edge> { - private final Edge outEdge = new Edge(); - - @Override - public Edge map(Tuple2 t) throws Exception { - outEdge.copyVerticesFromTuple2(t); - return outEdge; - } - } - - /** - * Projects an edge (pair of vertices) such that the id of the first is smaller than the id of - * the second. - */ - private static class EdgeByIdProjector implements MapFunction { - - @Override - public Edge map(Edge inEdge) throws Exception { - - // flip vertices if necessary - if (inEdge.getFirstVertex() > inEdge.getSecondVertex()) { - inEdge.flipVertices(); - } - - return inEdge; - } - } - - /** - * Builds triads (triples of vertices) from pairs of edges that share a vertex. The first vertex - * of a triad is the shared vertex, the second and third vertex are ordered by vertexId. Assumes - * that input edges share the first vertex and are in ascending order of the second vertex. - */ - @ForwardedFields("0") - private static class TriadBuilder implements GroupReduceFunction { - private final List vertices = new ArrayList(); - private final Triad outTriad = new Triad(); - - @Override - public void reduce(Iterable edgesIter, Collector out) throws Exception { - - final Iterator edges = edgesIter.iterator(); - - // clear vertex list - vertices.clear(); - - // read first edge - Edge firstEdge = edges.next(); - outTriad.setFirstVertex(firstEdge.getFirstVertex()); - vertices.add(firstEdge.getSecondVertex()); - - // build and emit triads - while (edges.hasNext()) { - Integer higherVertexId = edges.next().getSecondVertex(); - - // combine vertex with all previously read vertices - for (Integer lowerVertexId : vertices) { - outTriad.setSecondVertex(lowerVertexId); - outTriad.setThirdVertex(higherVertexId); - out.collect(outTriad); - } - vertices.add(higherVertexId); - } - } - } - - /** Filters triads (three vertices connected by two edges) without a closing third edge. */ - private static class TriadFilter implements JoinFunction { - - @Override - public Triad join(Triad triad, Edge edge) throws Exception { - return triad; - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/PageRank.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/PageRank.java deleted file mode 100644 index 5f58210a888ab..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/PageRank.java +++ /dev/null @@ -1,291 +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.graph; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -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.functions.FunctionAnnotation.ForwardedFields; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.examples.java.graph.util.PageRankData; -import org.apache.flink.util.Collector; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; - -import static org.apache.flink.api.java.aggregation.Aggregations.SUM; -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * A basic implementation of the Page Rank algorithm using a bulk iteration. - * - *

This implementation requires a set of pages and a set of directed links as input and works as - * follows.
- * In each iteration, the rank of every page is evenly distributed to all pages it points to. Each - * page collects the partial ranks of all pages that point to it, sums them up, and applies a - * dampening factor to the sum. The result is the new rank of the page. A new iteration is started - * with the new ranks of all pages. This implementation terminates after a fixed number of - * iterations.
- * This is the Wikipedia entry for the Page Rank - * algorithm. - * - *

Input files are plain text files and must be formatted as follows: - * - *

    - *
  • Pages represented as an (long) ID separated by new-line characters.
    - * For example "1\n2\n12\n42\n63" gives five pages with IDs 1, 2, 12, 42, and 63. - *
  • Links are represented as pairs of page IDs which are separated by space characters. Links - * are separated by new-line characters.
    - * For example "1 2\n2 12\n1 12\n42 63" gives four (directed) links (1)->(2), - * (2)->(12), (1)->(12), and (42)->(63).
    - * For this simple implementation it is required that each page has at least one incoming and - * one outgoing link (a page can point to itself). - *
- * - *

Usage: - * PageRankBasic --pages <path> --links <path> --output <path> --numPages <n> --iterations <n> - *
- * If no parameters are provided, the program is run with default data from {@link - * org.apache.flink.examples.java.graph.util.PageRankData} and 10 iterations. - * - *

This example shows how to use: - * - *

    - *
  • Bulk Iterations - *
  • Default Join - *
  • Configure user-defined functions using constructor parameters. - *
- * - *

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 PageRank { - - private static final Logger LOGGER = LoggerFactory.getLogger(PageRank.class); - private static final double DAMPENING_FACTOR = 0.85; - private static final double EPSILON = 0.0001; - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - ParameterTool params = ParameterTool.fromArgs(args); - - final int numPages = params.getInt("numPages", PageRankData.getNumberOfPages()); - final int maxIterations = params.getInt("iterations", 10); - - // set up execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // make the parameters available to the web ui - env.getConfig().setGlobalJobParameters(params); - - // get input data - DataSet pagesInput = getPagesDataSet(env, params); - DataSet> linksInput = getLinksDataSet(env, params); - - // assign initial rank to pages - DataSet> pagesWithRanks = - pagesInput.map(new RankAssigner((1.0d / numPages))); - - // build adjacency list from link input - DataSet> adjacencyListInput = - linksInput.groupBy(0).reduceGroup(new BuildOutgoingEdgeList()); - - // set iterative data set - IterativeDataSet> iteration = pagesWithRanks.iterate(maxIterations); - - DataSet> newRanks = - iteration - // join pages with outgoing edges and distribute rank - .join(adjacencyListInput) - .where(0) - .equalTo(0) - .flatMap(new JoinVertexWithEdgesMatch()) - // collect and sum ranks - .groupBy(0) - .aggregate(SUM, 1) - // apply dampening factor - .map(new Dampener(DAMPENING_FACTOR, numPages)); - - DataSet> finalPageRanks = - iteration.closeWith( - newRanks, - newRanks.join(iteration) - .where(0) - .equalTo(0) - // termination condition - .filter(new EpsilonFilter())); - - // emit result - if (params.has("output")) { - finalPageRanks.writeAsCsv(params.get("output"), "\n", " "); - // execute program - env.execute("Basic Page Rank Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - finalPageRanks.print(); - } - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** A map function that assigns an initial rank to all pages. */ - public static final class RankAssigner implements MapFunction> { - Tuple2 outPageWithRank; - - public RankAssigner(double rank) { - this.outPageWithRank = new Tuple2(-1L, rank); - } - - @Override - public Tuple2 map(Long page) { - outPageWithRank.f0 = page; - return outPageWithRank; - } - } - - /** - * A reduce function that takes a sequence of edges and builds the adjacency list for the vertex - * where the edges originate. Run as a pre-processing step. - */ - @ForwardedFields("0") - public static final class BuildOutgoingEdgeList - implements GroupReduceFunction, Tuple2> { - - private final ArrayList neighbors = new ArrayList(); - - @Override - public void reduce( - Iterable> values, Collector> out) { - neighbors.clear(); - Long id = 0L; - - for (Tuple2 n : values) { - id = n.f0; - neighbors.add(n.f1); - } - out.collect( - new Tuple2(id, neighbors.toArray(new Long[neighbors.size()]))); - } - } - - /** Join function that distributes a fraction of a vertex's rank to all neighbors. */ - public static final class JoinVertexWithEdgesMatch - implements FlatMapFunction< - Tuple2, Tuple2>, Tuple2> { - - @Override - public void flatMap( - Tuple2, Tuple2> value, - Collector> out) { - Long[] neighbors = value.f1.f1; - double rank = value.f0.f1; - double rankToDistribute = rank / ((double) neighbors.length); - - for (Long neighbor : neighbors) { - out.collect(new Tuple2(neighbor, rankToDistribute)); - } - } - } - - /** The function that applies the page rank dampening formula. */ - @ForwardedFields("0") - public static final class Dampener - implements MapFunction, Tuple2> { - - private final double dampening; - private final double randomJump; - - public Dampener(double dampening, double numVertices) { - this.dampening = dampening; - this.randomJump = (1 - dampening) / numVertices; - } - - @Override - public Tuple2 map(Tuple2 value) { - value.f1 = (value.f1 * dampening) + randomJump; - return value; - } - } - - /** Filter that filters vertices where the rank difference is below a threshold. */ - public static final class EpsilonFilter - implements FilterFunction, Tuple2>> { - - @Override - public boolean filter(Tuple2, Tuple2> value) { - return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON; - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static DataSet getPagesDataSet(ExecutionEnvironment env, ParameterTool params) { - if (params.has("pages")) { - return env.readCsvFile(params.get("pages")) - .fieldDelimiter(" ") - .lineDelimiter("\n") - .types(Long.class) - .map( - new MapFunction, Long>() { - @Override - public Long map(Tuple1 v) { - return v.f0; - } - }); - } else { - System.out.println("Executing PageRank example with default pages data set."); - System.out.println("Use --pages to specify file input."); - return PageRankData.getDefaultPagesDataSet(env); - } - } - - private static DataSet> getLinksDataSet( - ExecutionEnvironment env, ParameterTool params) { - if (params.has("links")) { - return env.readCsvFile(params.get("links")) - .fieldDelimiter(" ") - .lineDelimiter("\n") - .types(Long.class, Long.class); - } else { - System.out.println("Executing PageRank example with default links data set."); - System.out.println("Use --links to specify file input."); - return PageRankData.getDefaultEdgeDataSet(env); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java deleted file mode 100644 index 291828a39fd75..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/TransitiveClosureNaive.java +++ /dev/null @@ -1,167 +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.graph; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.examples.java.graph.util.ConnectedComponentsData; -import org.apache.flink.util.Collector; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashSet; -import java.util.Set; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * The transitive closure of a graph contains an edge for each pair of vertices which are endpoints - * of at least one path in the graph. - * - *

This algorithm is implemented using a delta iteration. The transitive closure solution set is - * grown in each step by joining the workset of newly discovered path endpoints with the original - * graph edges and discarding previously discovered path endpoints (already in the solution set). - * - *

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 TransitiveClosureNaive { - - private static final Logger LOGGER = LoggerFactory.getLogger(TransitiveClosureNaive.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(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - final int maxIterations = params.getInt("iterations", 10); - - DataSet> edges; - if (params.has("edges")) { - edges = - env.readCsvFile(params.get("edges")) - .fieldDelimiter(" ") - .types(Long.class, Long.class); - } else { - System.out.println( - "Executing TransitiveClosureNaive example with default edges data set."); - System.out.println("Use --edges to specify file input."); - edges = ConnectedComponentsData.getDefaultEdgeDataSet(env); - } - - IterativeDataSet> paths = edges.iterate(maxIterations); - - DataSet> nextPaths = - paths.join(edges) - .where(1) - .equalTo(0) - .with( - new JoinFunction< - Tuple2, - Tuple2, - Tuple2>() { - @Override - /** - * left: Path (z,x) - x is reachable by z right: Edge (x,y) - - * edge x-->y exists out: Path (z,y) - y is reachable by z - */ - public Tuple2 join( - Tuple2 left, Tuple2 right) - throws Exception { - return new Tuple2(left.f0, right.f1); - } - }) - .withForwardedFieldsFirst("0") - .withForwardedFieldsSecond("1") - .union(paths) - .groupBy(0, 1) - .reduceGroup( - new GroupReduceFunction, Tuple2>() { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - out.collect(values.iterator().next()); - } - }) - .withForwardedFields("0;1"); - - DataSet> newPaths = - paths.coGroup(nextPaths) - .where(0) - .equalTo(0) - .with( - new CoGroupFunction< - Tuple2, - Tuple2, - Tuple2>() { - Set> prevSet = - new HashSet>(); - - @Override - public void coGroup( - Iterable> prevPaths, - Iterable> nextPaths, - Collector> out) - throws Exception { - for (Tuple2 prev : prevPaths) { - prevSet.add(prev); - } - for (Tuple2 next : nextPaths) { - if (!prevSet.contains(next)) { - out.collect(next); - } - } - } - }) - .withForwardedFieldsFirst("0") - .withForwardedFieldsSecond("0"); - - DataSet> transitiveClosure = paths.closeWith(nextPaths, newPaths); - - // emit result - if (params.has("output")) { - transitiveClosure.writeAsCsv(params.get("output"), "\n", " "); - - // execute program explicitly, because file sinks are lazy - env.execute("Transitive Closure Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - transitiveClosure.print(); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java deleted file mode 100644 index 7635f460d7c8d..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/ConnectedComponentsData.java +++ /dev/null @@ -1,71 +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.graph.util; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; - -import java.util.LinkedList; -import java.util.List; - -/** - * Provides the default data sets used for the Connected Components example program. The default - * data sets are used, if no parameters are given to the program. - */ -public class ConnectedComponentsData { - - public static final long[] VERTICES = - new long[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}; - - public static DataSet getDefaultVertexDataSet(ExecutionEnvironment env) { - List verticesList = new LinkedList(); - for (long vertexId : VERTICES) { - verticesList.add(vertexId); - } - return env.fromCollection(verticesList); - } - - public static final Object[][] EDGES = - new Object[][] { - new Object[] {1L, 2L}, - new Object[] {2L, 3L}, - new Object[] {2L, 4L}, - new Object[] {3L, 5L}, - new Object[] {6L, 7L}, - new Object[] {8L, 9L}, - new Object[] {8L, 10L}, - new Object[] {5L, 11L}, - new Object[] {11L, 12L}, - new Object[] {10L, 13L}, - new Object[] {9L, 14L}, - new Object[] {13L, 14L}, - new Object[] {1L, 15L}, - new Object[] {16L, 1L} - }; - - public static DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { - - List> edgeList = new LinkedList>(); - for (Object[] edge : EDGES) { - edgeList.add(new Tuple2((Long) edge[0], (Long) edge[1])); - } - return env.fromCollection(edgeList); - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java deleted file mode 100644 index 6500ab8074556..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesData.java +++ /dev/null @@ -1,58 +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.graph.util; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.examples.java.graph.util.EnumTrianglesDataTypes.Edge; - -import java.util.ArrayList; -import java.util.List; - -/** - * Provides the default data sets used for the Triangle Enumeration example programs. The default - * data sets are used, if no parameters are given to the program. - */ -public class EnumTrianglesData { - - public static final Object[][] EDGES = { - {1, 2}, - {1, 3}, - {1, 4}, - {1, 5}, - {2, 3}, - {2, 5}, - {3, 4}, - {3, 7}, - {3, 8}, - {5, 6}, - {7, 8} - }; - - public static DataSet getDefaultEdgeDataSet( - ExecutionEnvironment env) { - - List edges = new ArrayList(); - for (Object[] e : EDGES) { - edges.add(new Edge((Integer) e[0], (Integer) e[1])); - } - - return env.fromCollection(edges); - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java deleted file mode 100644 index 754626c7e3d28..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/EnumTrianglesDataTypes.java +++ /dev/null @@ -1,148 +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.graph.util; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; - -/** The data classes for EnumTriangles. */ -public class EnumTrianglesDataTypes { - - /** A POJO storing two vertex IDs. */ - public static class Edge extends Tuple2 { - private static final long serialVersionUID = 1L; - - public static final int V1 = 0; - public static final int V2 = 1; - - public Edge() {} - - public Edge(final Integer v1, final Integer v2) { - this.setFirstVertex(v1); - this.setSecondVertex(v2); - } - - public Integer getFirstVertex() { - return this.getField(V1); - } - - public Integer getSecondVertex() { - return this.getField(V2); - } - - public void setFirstVertex(final Integer vertex1) { - this.setField(vertex1, V1); - } - - public void setSecondVertex(final Integer vertex2) { - this.setField(vertex2, V2); - } - - public void copyVerticesFromTuple2(Tuple2 t) { - this.setFirstVertex(t.f0); - this.setSecondVertex(t.f1); - } - - public void copyVerticesFromEdgeWithDegrees(EdgeWithDegrees ewd) { - this.setFirstVertex(ewd.getFirstVertex()); - this.setSecondVertex(ewd.getSecondVertex()); - } - - public void flipVertices() { - Integer tmp = this.getFirstVertex(); - this.setFirstVertex(this.getSecondVertex()); - this.setSecondVertex(tmp); - } - } - - /** A POJO storing three vertex IDs. */ - public static class Triad extends Tuple3 { - private static final long serialVersionUID = 1L; - - public static final int V1 = 0; - public static final int V2 = 1; - public static final int V3 = 2; - - public Triad() {} - - public void setFirstVertex(final Integer vertex1) { - this.setField(vertex1, V1); - } - - public void setSecondVertex(final Integer vertex2) { - this.setField(vertex2, V2); - } - - public void setThirdVertex(final Integer vertex3) { - this.setField(vertex3, V3); - } - } - - /** A POJO storing two vertex IDs with degree. */ - public static class EdgeWithDegrees extends Tuple4 { - private static final long serialVersionUID = 1L; - - public static final int V1 = 0; - public static final int V2 = 1; - public static final int D1 = 2; - public static final int D2 = 3; - - public EdgeWithDegrees() {} - - public Integer getFirstVertex() { - return this.getField(V1); - } - - public Integer getSecondVertex() { - return this.getField(V2); - } - - public Integer getFirstDegree() { - return this.getField(D1); - } - - public Integer getSecondDegree() { - return this.getField(D2); - } - - public void setFirstVertex(final Integer vertex1) { - this.setField(vertex1, V1); - } - - public void setSecondVertex(final Integer vertex2) { - this.setField(vertex2, V2); - } - - public void setFirstDegree(final Integer degree1) { - this.setField(degree1, D1); - } - - public void setSecondDegree(final Integer degree2) { - this.setField(degree2, D2); - } - - public void copyFrom(final EdgeWithDegrees edge) { - this.setFirstVertex(edge.getFirstVertex()); - this.setSecondVertex(edge.getSecondVertex()); - this.setFirstDegree(edge.getFirstDegree()); - this.setSecondDegree(edge.getSecondDegree()); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java deleted file mode 100644 index 6a08bec31c6b3..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/graph/util/PageRankData.java +++ /dev/null @@ -1,84 +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.graph.util; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; - -import java.util.ArrayList; -import java.util.List; - -/** - * Provides the default data sets used for the PageRank example program. The default data sets are - * used, if no parameters are given to the program. - */ -public class PageRankData { - - public static final Object[][] EDGES = { - {1L, 2L}, - {1L, 15L}, - {2L, 3L}, - {2L, 4L}, - {2L, 5L}, - {2L, 6L}, - {2L, 7L}, - {3L, 13L}, - {4L, 2L}, - {5L, 11L}, - {5L, 12L}, - {6L, 1L}, - {6L, 7L}, - {6L, 8L}, - {7L, 1L}, - {7L, 8L}, - {8L, 1L}, - {8L, 9L}, - {8L, 10L}, - {9L, 14L}, - {9L, 1L}, - {10L, 1L}, - {10L, 13L}, - {11L, 12L}, - {11L, 1L}, - {12L, 1L}, - {13L, 14L}, - {14L, 12L}, - {15L, 1L}, - }; - - private static int numPages = 15; - - public static DataSet> getDefaultEdgeDataSet(ExecutionEnvironment env) { - - List> edges = new ArrayList>(); - for (Object[] e : EDGES) { - edges.add(new Tuple2((Long) e[0], (Long) e[1])); - } - return env.fromCollection(edges); - } - - public static DataSet getDefaultPagesDataSet(ExecutionEnvironment env) { - return env.generateSequence(1, 15); - } - - public static int getNumberOfPages() { - return numPages; - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/misc/CollectionExecutionExample.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/misc/CollectionExecutionExample.java deleted file mode 100644 index 60d0a951ac218..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/misc/CollectionExecutionExample.java +++ /dev/null @@ -1,117 +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.misc; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * This example shows how to use the collection based execution of Flink. - * - *

The collection based execution is a local mode that is not using the full Flink runtime. - * DataSet transformations are executed on Java collections. - * - *

See the "Local Execution" section in the documentation for more details: - * https://flink.apache.org/docs/latest/apis/local_execution.html - * - *

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 CollectionExecutionExample { - - private static final Logger LOGGER = LoggerFactory.getLogger(CollectionExecutionExample.class); - - /** POJO class representing a user. */ - public static class User { - public int userIdentifier; - public String name; - - public User() {} - - public User(int userIdentifier, String name) { - this.userIdentifier = userIdentifier; - this.name = name; - } - - public String toString() { - return "User{userIdentifier=" + userIdentifier + " name=" + name + "}"; - } - } - - /** POJO for an EMail. */ - public static class EMail { - public int userId; - public String subject; - public String body; - - public EMail() {} - - public EMail(int userId, String subject, String body) { - this.userId = userId; - this.subject = subject; - this.body = body; - } - - public String toString() { - return "eMail{userId=" + userId + " subject=" + subject + " body=" + body + "}"; - } - } - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - // initialize a new Collection-based execution environment - final ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - // create objects for users and emails - User[] usersArray = {new User(1, "Peter"), new User(2, "John"), new User(3, "Bill")}; - - EMail[] emailsArray = { - new EMail(1, "Re: Meeting", "How about 1pm?"), - new EMail(1, "Re: Meeting", "Sorry, I'm not available"), - new EMail(3, "Re: Re: Project proposal", "Give me a few more days to think about it.") - }; - - // convert objects into a DataSet - DataSet users = env.fromElements(usersArray); - DataSet emails = env.fromElements(emailsArray); - - // join the two DataSets - DataSet> joined = - users.join(emails).where("userIdentifier").equalTo("userId"); - - // retrieve the resulting Tuple2 elements into a ArrayList. - List> result = joined.collect(); - - // Do some work with the resulting ArrayList (=Collection). - for (Tuple2 t : result) { - System.err.println("Result = " + t); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java deleted file mode 100644 index a71aceee84ef9..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/misc/PiEstimation.java +++ /dev/null @@ -1,103 +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.misc; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * Estimates the value of Pi using the Monte Carlo method. The area of a circle is Pi * R^2, R being - * the radius of the circle The area of a square is 4 * R^2, where the length of the square's edge - * is 2*R. - * - *

Thus Pi = 4 * (area of circle / area of square). - * - *

The idea is to find a way to estimate the circle to square area ratio. The Monte Carlo method - * suggests collecting random points (within the square) and then counting the number of points that - * fall within the circle - * - *

{@code
- * x = Math.random()
- * y = Math.random()
- *
- * x * x + y * y < 1
- * }
- * - *

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 PiEstimation implements java.io.Serializable { - - private static final Logger LOGGER = LoggerFactory.getLogger(PiEstimation.class); - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final long numSamples = args.length > 0 ? Long.parseLong(args[0]) : 1000000; - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // count how many of the samples would randomly fall into - // the unit circle - DataSet count = - env.generateSequence(1, numSamples).map(new Sampler()).reduce(new SumReducer()); - - long theCount = count.collect().get(0); - - System.out.println("We estimate Pi to be: " + (theCount * 4.0 / numSamples)); - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Sampler randomly emits points that fall within a square of edge x * y. It calculates the - * distance to the center of a virtually centered circle of radius x = y = 1 If the distance is - * less than 1, then and only then does it returns a 1. - */ - public static class Sampler implements MapFunction { - - @Override - public Long map(Long value) { - double x = Math.random(); - double y = Math.random(); - return (x * x + y * y) < 1 ? 1L : 0L; - } - } - - /** Simply sums up all long values. */ - public static final class SumReducer implements ReduceFunction { - - @Override - public Long reduce(Long value1, Long value2) { - return value1 + value2; - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java deleted file mode 100644 index e493a65285c95..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java +++ /dev/null @@ -1,262 +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.relational; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.utils.ParameterTool; - -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * This program filters lines from a CSV file with empty fields. In doing so, it counts the number - * of empty fields per column within a CSV file using a custom accumulator for vectors. In this - * context, empty fields are those, that at most contain whitespace characters like space and tab. - * - *

The input file is a plain text CSV file with the semicolon as field separator and double - * quotes as field delimiters and three columns. See {@link #getDataSet(ExecutionEnvironment, - * ParameterTool)} for configuration. - * - *

Usage: EmptyFieldsCountAccumulator --input <path> --output <path> - *
- * - *

This example shows how to use: - * - *

    - *
  • custom accumulators - *
  • tuple data types - *
  • inline-defined functions - *
  • naming large tuple types - *
- * - *

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 EmptyFieldsCountAccumulator { - - private static final Logger LOGGER = LoggerFactory.getLogger(EmptyFieldsCountAccumulator.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - private static final String EMPTY_FIELD_ACCUMULATOR = "empty-fields"; - - public static void main(final String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final ParameterTool params = ParameterTool.fromArgs(args); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // get the data set - final DataSet file = getDataSet(env, params); - - // filter lines with empty fields - final DataSet filteredLines = file.filter(new EmptyFieldFilter()); - - // Here, we could do further processing with the filtered lines... - JobExecutionResult result; - // output the filtered lines - if (params.has("output")) { - filteredLines.writeAsCsv(params.get("output")); - // execute program - result = env.execute("Accumulator example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - filteredLines.print(); - result = env.getLastJobExecutionResult(); - } - - // get the accumulator result via its registration key - final List emptyFields = result.getAccumulatorResult(EMPTY_FIELD_ACCUMULATOR); - System.out.format("Number of detected empty fields per column: %s\n", emptyFields); - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - @SuppressWarnings("unchecked") - private static DataSet getDataSet( - ExecutionEnvironment env, ParameterTool params) { - if (params.has("input")) { - return env.readCsvFile(params.get("input")) - .fieldDelimiter(";") - .pojoType(StringTriple.class); - } else { - System.out.println( - "Executing EmptyFieldsCountAccumulator example with default input data set."); - System.out.println("Use --input to specify file input."); - return env.fromCollection(getExampleInputTuples()); - } - } - - private static Collection getExampleInputTuples() { - Collection inputTuples = new ArrayList(); - inputTuples.add(new StringTriple("John", "Doe", "Foo Str.")); - inputTuples.add(new StringTriple("Joe", "Johnson", "")); - inputTuples.add(new StringTriple(null, "Kate Morn", "Bar Blvd.")); - inputTuples.add(new StringTriple("Tim", "Rinny", "")); - inputTuples.add(new StringTriple("Alicia", "Jackson", " ")); - return inputTuples; - } - - /** - * This function filters all incoming tuples that have one or more empty fields. In doing so, it - * also counts the number of empty fields per attribute with an accumulator (registered under - * {@link EmptyFieldsCountAccumulator#EMPTY_FIELD_ACCUMULATOR}). - */ - public static final class EmptyFieldFilter extends RichFilterFunction { - - // create a new accumulator in each filter function instance - // accumulators can be merged later on - private final VectorAccumulator emptyFieldCounter = new VectorAccumulator(); - - @Override - public void open(final OpenContext openContext) throws Exception { - super.open(openContext); - - // register the accumulator instance - getRuntimeContext().addAccumulator(EMPTY_FIELD_ACCUMULATOR, this.emptyFieldCounter); - } - - @Override - public boolean filter(final StringTriple t) { - boolean containsEmptyFields = false; - - // iterate over the tuple fields looking for empty ones - for (int pos = 0; pos < t.getArity(); pos++) { - - final String field = t.getField(pos); - if (field == null || field.trim().isEmpty()) { - containsEmptyFields = true; - - // if an empty field is encountered, update the - // accumulator - this.emptyFieldCounter.add(pos); - } - } - - return !containsEmptyFields; - } - } - - /** - * This accumulator maintains a vector of counts. Calling {@link #add(Integer)} increments the - * n-th vector component. The size of the vector is automatically managed. - */ - public static class VectorAccumulator implements Accumulator> { - - /** Stores the accumulated vector components. */ - private final ArrayList resultVector; - - public VectorAccumulator() { - this(new ArrayList()); - } - - public VectorAccumulator(ArrayList resultVector) { - this.resultVector = resultVector; - } - - /** Increases the result vector component at the specified position by 1. */ - @Override - public void add(Integer position) { - updateResultVector(position, 1); - } - - /** - * Increases the result vector component at the specified position by the specified delta. - */ - private void updateResultVector(int position, int delta) { - // inflate the vector to contain the given position - while (this.resultVector.size() <= position) { - this.resultVector.add(0); - } - - // increment the component value - final int component = this.resultVector.get(position); - this.resultVector.set(position, component + delta); - } - - @Override - public ArrayList getLocalValue() { - return this.resultVector; - } - - @Override - public void resetLocal() { - // clear the result vector if the accumulator instance shall be reused - this.resultVector.clear(); - } - - @Override - public void merge(final Accumulator> other) { - // merge two vector accumulators by adding their up their vector components - final List otherVector = other.getLocalValue(); - for (int index = 0; index < otherVector.size(); index++) { - updateResultVector(index, otherVector.get(index)); - } - } - - @Override - public Accumulator> clone() { - return new VectorAccumulator(new ArrayList(resultVector)); - } - - @Override - public String toString() { - return StringUtils.join(resultVector, ','); - } - } - - /** - * It is recommended to use POJOs (Plain old Java objects) instead of TupleX for data types with - * many fields. Also, POJOs can be used to give large Tuple-types a name. Source - * (docs) - */ - public static class StringTriple extends Tuple3 { - - public StringTriple() {} - - public StringTriple(String f0, String f1, String f2) { - super(f0, f1, f2); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java deleted file mode 100644 index f3aa98259ad17..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java +++ /dev/null @@ -1,234 +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.relational; - -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.aggregation.Aggregations; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple6; -import org.apache.flink.api.java.utils.ParameterTool; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * This program implements a modified version of the TPC-H query 10. The original query can be found - * at http://www.tpc.org/tpch/spec/tpch2.16.0.pdf - * (page 45). - * - *

This program implements the following SQL equivalent: - * - *

{@code
- * SELECT
- *        c_custkey,
- *        c_name,
- *        c_address,
- *        n_name,
- *        c_acctbal
- *        SUM(l_extendedprice * (1 - l_discount)) AS revenue,
- * FROM
- *        customer,
- *        orders,
- *        lineitem,
- *        nation
- * WHERE
- *        c_custkey = o_custkey
- *        AND l_orderkey = o_orderkey
- *        AND YEAR(o_orderdate) > '1990'
- *        AND l_returnflag = 'R'
- *        AND c_nationkey = n_nationkey
- * GROUP BY
- *        c_custkey,
- *        c_name,
- *        c_acctbal,
- *        n_name,
- *        c_address
- * }
- * - *

Compared to the original TPC-H query this version does not print c_phone and c_comment, only - * filters by years greater than 1990 instead of a period of 3 months, and does not sort the result - * by revenue. - * - *

Input files are plain text CSV files using the pipe character ('|') as field separator as - * generated by the TPC-H data generator which is available at http://www.tpc.org/tpch/. - * - *

Usage: - * TPCHQuery10 --customer <path> --orders <path> --lineitem<path> --nation <path> --output <path> - *
- * - *

This example shows how to use: - * - *

    - *
  • tuple data types - *
  • inline-defined functions - *
  • projection and join projection - *
  • built-in aggregation functions - *
- * - *

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 TPCHQuery10 { - - private static final Logger LOGGER = LoggerFactory.getLogger(TPCHQuery10.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final ParameterTool params = ParameterTool.fromArgs(args); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - if (!params.has("customer") - && !params.has("orders") - && !params.has("lineitem") - && !params.has("nation")) { - System.err.println( - " This program expects data from the TPC-H benchmark as input data."); - System.err.println(" Due to legal restrictions, we can not ship generated data."); - System.err.println( - " You can find the TPC-H data generator at http://www.tpc.org/tpch/."); - System.err.println( - " Usage: TPCHQuery10 --customer --orders --lineitem --nation [--output ]"); - return; - } - - // get customer data set: (custkey, name, address, nationkey, acctbal) - DataSet> customers = - getCustomerDataSet(env, params.get("customer")); - // get orders data set: (orderkey, custkey, orderdate) - DataSet> orders = - getOrdersDataSet(env, params.get("orders")); - // get lineitem data set: (orderkey, extendedprice, discount, returnflag) - DataSet> lineitems = - getLineitemDataSet(env, params.get("lineitem")); - // get nation data set: (nationkey, name) - DataSet> nations = getNationsDataSet(env, params.get("nation")); - - // orders filtered by year: (orderkey, custkey) - DataSet> ordersFilteredByYear = - // filter by year - orders.filter(order -> Integer.parseInt(order.f2.substring(0, 4)) > 1990) - // project fields out that are no longer required - .project(0, 1); - - // lineitems filtered by flag: (orderkey, revenue) - DataSet> lineitemsFilteredByFlag = - // filter by flag - lineitems - .filter(lineitem -> lineitem.f3.equals("R")) - // compute revenue and project out return flag - // revenue per item = l_extendedprice * (1 - l_discount) - .map(lineitem -> new Tuple2<>(lineitem.f0, lineitem.f1 * (1 - lineitem.f2))) - .returns(Types.TUPLE(Types.INT, Types.DOUBLE)); // for lambda with generics - - // join orders with lineitems: (custkey, revenue) - DataSet> revenueByCustomer = - ordersFilteredByYear - .joinWithHuge(lineitemsFilteredByFlag) - .where(0) - .equalTo(0) - .projectFirst(1) - .projectSecond(1); - - revenueByCustomer = revenueByCustomer.groupBy(0).aggregate(Aggregations.SUM, 1); - - // join customer with nation (custkey, name, address, nationname, acctbal) - DataSet> customerWithNation = - customers - .joinWithTiny(nations) - .where(3) - .equalTo(0) - .projectFirst(0, 1, 2) - .projectSecond(1) - .projectFirst(4); - - // join customer (with nation) with revenue (custkey, name, address, nationname, acctbal, - // revenue) - DataSet> result = - customerWithNation - .join(revenueByCustomer) - .where(0) - .equalTo(0) - .projectFirst(0, 1, 2, 3, 4) - .projectSecond(1); - - // emit result - if (params.has("output")) { - result.writeAsCsv(params.get("output"), "\n", "|"); - // execute program - env.execute("TPCH Query 10 Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - result.print(); - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static DataSet> getCustomerDataSet( - ExecutionEnvironment env, String customerPath) { - return env.readCsvFile(customerPath) - .fieldDelimiter("|") - .includeFields("11110100") - .types(Integer.class, String.class, String.class, Integer.class, Double.class); - } - - private static DataSet> getOrdersDataSet( - ExecutionEnvironment env, String ordersPath) { - return env.readCsvFile(ordersPath) - .fieldDelimiter("|") - .includeFields("110010000") - .types(Integer.class, Integer.class, String.class); - } - - private static DataSet> getLineitemDataSet( - ExecutionEnvironment env, String lineitemPath) { - return env.readCsvFile(lineitemPath) - .fieldDelimiter("|") - .includeFields("1000011010000000") - .types(Integer.class, Double.class, Double.class, String.class); - } - - private static DataSet> getNationsDataSet( - ExecutionEnvironment env, String nationPath) { - return env.readCsvFile(nationPath) - .fieldDelimiter("|") - .includeFields("1100") - .types(Integer.class, String.class); - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java deleted file mode 100644 index cb8d7786c9f22..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java +++ /dev/null @@ -1,333 +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.relational; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.JoinFunction; -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.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.utils.ParameterTool; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.text.DateFormat; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Date; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * This program implements a modified version of the TPC-H query 3. The example demonstrates how to - * assign names to fields by extending the Tuple class. The original query can be found at http://www.tpc.org/tpch/spec/tpch2.16.0.pdf - * (page 29). - * - *

This program implements the following SQL equivalent: - * - *

{@code
- * SELECT
- *      l_orderkey,
- *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
- *      o_orderdate,
- *      o_shippriority
- * FROM customer,
- *      orders,
- *      lineitem
- * WHERE
- *      c_mktsegment = '[SEGMENT]'
- *      AND c_custkey = o_custkey
- *      AND l_orderkey = o_orderkey
- *      AND o_orderdate < date '[DATE]'
- *      AND l_shipdate > date '[DATE]'
- * GROUP BY
- *      l_orderkey,
- *      o_orderdate,
- *      o_shippriority;
- * }
- * - *

Compared to the original TPC-H query this version does not sort the result by revenue and - * orderdate. - * - *

Input files are plain text CSV files using the pipe character ('|') as field separator as - * generated by the TPC-H data generator which is available at http://www.tpc.org/tpch/. - * - *

Usage: - * TPCHQuery3 --lineitem<path> --customer <path> --orders<path> --output <path> - *
- * - *

This example shows how to use: - * - *

    - *
  • custom data type derived from tuple data types - *
  • inline-defined functions - *
  • build-in aggregation functions - *
- * - *

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 TPCHQuery3 { - - private static final Logger LOGGER = LoggerFactory.getLogger(TPCHQuery3.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final ParameterTool params = ParameterTool.fromArgs(args); - - if (!params.has("lineitem") && !params.has("customer") && !params.has("orders")) { - System.err.println( - " This program expects data from the TPC-H benchmark as input data."); - System.err.println(" Due to legal restrictions, we can not ship generated data."); - System.out.println( - " You can find the TPC-H data generator at http://www.tpc.org/tpch/."); - System.out.println( - " Usage: TPCHQuery3 --lineitem --customer --orders [--output ]"); - return; - } - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - env.getConfig().setGlobalJobParameters(params); - - // get input data - DataSet lineitems = getLineitemDataSet(env, params.get("lineitem")); - DataSet customers = getCustomerDataSet(env, params.get("customer")); - DataSet orders = getOrdersDataSet(env, params.get("orders")); - - // Filter market segment "AUTOMOBILE" - customers = - customers.filter( - new FilterFunction() { - @Override - public boolean filter(Customer c) { - return c.getMktsegment().equals("AUTOMOBILE"); - } - }); - - // Filter all Orders with o_orderdate < 12.03.1995 - orders = - orders.filter( - new FilterFunction() { - private final DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); - private final Date date = format.parse("1995-03-12"); - - @Override - public boolean filter(Order o) throws ParseException { - return format.parse(o.getOrderdate()).before(date); - } - }); - - // Filter all Lineitems with l_shipdate > 12.03.1995 - lineitems = - lineitems.filter( - new FilterFunction() { - private final DateFormat format = new SimpleDateFormat("yyyy-MM-dd"); - private final Date date = format.parse("1995-03-12"); - - @Override - public boolean filter(Lineitem l) throws ParseException { - return format.parse(l.getShipdate()).after(date); - } - }); - - // Join customers with orders and package them into a ShippingPriorityItem - DataSet customerWithOrders = - customers - .join(orders) - .where(0) - .equalTo(1) - .with( - new JoinFunction() { - @Override - public ShippingPriorityItem join(Customer c, Order o) { - return new ShippingPriorityItem( - o.getOrderKey(), - 0.0, - o.getOrderdate(), - o.getShippriority()); - } - }); - - // Join the last join result with Lineitems - DataSet result = - customerWithOrders - .join(lineitems) - .where(0) - .equalTo(0) - .with( - new JoinFunction< - ShippingPriorityItem, Lineitem, ShippingPriorityItem>() { - @Override - public ShippingPriorityItem join( - ShippingPriorityItem i, Lineitem l) { - i.setRevenue(l.getExtendedprice() * (1 - l.getDiscount())); - return i; - } - }) - // Group by l_orderkey, o_orderdate and o_shippriority and compute revenue - // sum - .groupBy(0, 2, 3) - .aggregate(Aggregations.SUM, 1); - - // emit result - if (params.has("output")) { - result.writeAsCsv(params.get("output"), "\n", "|"); - // execute program - env.execute("TPCH Query 3 Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - result.print(); - } - } - - // ************************************************************************* - // DATA TYPES - // ************************************************************************* - - /** Lineitem. */ - public static class Lineitem extends Tuple4 { - - public Long getOrderkey() { - return this.f0; - } - - public Double getDiscount() { - return this.f2; - } - - public Double getExtendedprice() { - return this.f1; - } - - public String getShipdate() { - return this.f3; - } - } - - /** Customer. */ - public static class Customer extends Tuple2 { - - public Long getCustKey() { - return this.f0; - } - - public String getMktsegment() { - return this.f1; - } - } - - /** Order. */ - public static class Order extends Tuple4 { - - public Long getOrderKey() { - return this.f0; - } - - public Long getCustKey() { - return this.f1; - } - - public String getOrderdate() { - return this.f2; - } - - public Long getShippriority() { - return this.f3; - } - } - - /** ShippingPriorityItem. */ - public static class ShippingPriorityItem extends Tuple4 { - - public ShippingPriorityItem() {} - - public ShippingPriorityItem( - Long orderkey, Double revenue, String orderdate, Long shippriority) { - this.f0 = orderkey; - this.f1 = revenue; - this.f2 = orderdate; - this.f3 = shippriority; - } - - public Long getOrderkey() { - return this.f0; - } - - public void setOrderkey(Long orderkey) { - this.f0 = orderkey; - } - - public Double getRevenue() { - return this.f1; - } - - public void setRevenue(Double revenue) { - this.f1 = revenue; - } - - public String getOrderdate() { - return this.f2; - } - - public Long getShippriority() { - return this.f3; - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static DataSet getLineitemDataSet( - ExecutionEnvironment env, String lineitemPath) { - return env.readCsvFile(lineitemPath) - .fieldDelimiter("|") - .includeFields("1000011000100000") - .tupleType(Lineitem.class); - } - - private static DataSet getCustomerDataSet( - ExecutionEnvironment env, String customerPath) { - return env.readCsvFile(customerPath) - .fieldDelimiter("|") - .includeFields("10000010") - .tupleType(Customer.class); - } - - private static DataSet getOrdersDataSet(ExecutionEnvironment env, String ordersPath) { - return env.readCsvFile(ordersPath) - .fieldDelimiter("|") - .includeFields("110010010") - .tupleType(Order.class); - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java deleted file mode 100644 index 1ddebcdd0117f..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java +++ /dev/null @@ -1,303 +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.relational; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.tuple.Tuple1; -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.relational.util.WebLogData; -import org.apache.flink.util.Collector; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * This program processes web logs and relational data. It implements the following relational - * query: - * - *

{@code
- * SELECT
- *       r.pageURL,
- *       r.pageRank,
- *       r.avgDuration
- * FROM documents d JOIN rankings r
- *                  ON d.url = r.url
- * WHERE CONTAINS(d.text, [keywords])
- *       AND r.rank > [rank]
- *       AND NOT EXISTS
- *           (
- *              SELECT * FROM Visits v
- *              WHERE v.destUrl = d.url
- *                    AND v.visitDate < [date]
- *           );
- * }
- * - *

Input files are plain text CSV files using the pipe character ('|') as field separator. The - * tables referenced in the query can be generated using the {@link - * org.apache.flink.examples.java.relational.util.WebLogDataGenerator} and have the following - * schemas - * - *

{@code
- * CREATE TABLE Documents (
- *                url VARCHAR(100) PRIMARY KEY,
- *                contents TEXT );
- *
- * CREATE TABLE Rankings (
- *                pageRank INT,
- *                pageURL VARCHAR(100) PRIMARY KEY,
- *                avgDuration INT );
- *
- * CREATE TABLE Visits (
- *                sourceIP VARCHAR(16),
- *                destURL VARCHAR(100),
- *                visitDate DATE,
- *                adRevenue FLOAT,
- *                userAgent VARCHAR(64),
- *                countryCode VARCHAR(3),
- *                languageCode VARCHAR(6),
- *                searchWord VARCHAR(32),
- *                duration INT );
- * }
- * - *

Usage: - * WebLogAnalysis --documents <path> --ranks <path> --visits <path> --result <path> - *
- * If no parameters are provided, the program is run with default data from {@link WebLogData}. - * - *

This example shows how to use: - * - *

    - *
  • tuple data types - *
  • projection and join projection - *
  • the CoGroup transformation for an anti-join - *
- * - *

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 WebLogAnalysis { - - private static final Logger LOGGER = LoggerFactory.getLogger(WebLogAnalysis.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final ParameterTool params = ParameterTool.fromArgs(args); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - env.getConfig().setGlobalJobParameters(params); - - // get input data - DataSet> documents = getDocumentsDataSet(env, params); - DataSet> ranks = getRanksDataSet(env, params); - DataSet> visits = getVisitsDataSet(env, params); - - // Retain documents with keywords - DataSet> filterDocs = documents.filter(new FilterDocByKeyWords()).project(0); - - // Filter ranks by minimum rank - DataSet> filterRanks = ranks.filter(new FilterByRank()); - - // Filter visits by visit date - DataSet> filterVisits = visits.filter(new FilterVisitsByDate()).project(0); - - // Join the filtered documents and ranks, i.e., get all URLs with min rank and keywords - DataSet> joinDocsRanks = - filterDocs.join(filterRanks).where(0).equalTo(1).projectSecond(0, 1, 2); - - // Anti-join urls with visits, i.e., retain all URLs which have NOT been visited in a - // certain time - DataSet> result = - joinDocsRanks.coGroup(filterVisits).where(1).equalTo(0).with(new AntiJoinVisits()); - - // emit result - if (params.has("output")) { - result.writeAsCsv(params.get("output"), "\n", "|"); - // execute program - env.execute("WebLogAnalysis Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - result.print(); - } - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** MapFunction that filters for documents that contain a certain set of keywords. */ - public static class FilterDocByKeyWords implements FilterFunction> { - - private static final String[] KEYWORDS = {" editors ", " oscillations "}; - - /** - * Filters for documents that contain all of the given keywords and projects the records on - * the URL field. - * - *

Output Format: 0: URL 1: DOCUMENT_TEXT - */ - @Override - public boolean filter(Tuple2 value) throws Exception { - // FILTER - // Only collect the document if all keywords are contained - String docText = value.f1; - for (String kw : KEYWORDS) { - if (!docText.contains(kw)) { - return false; - } - } - return true; - } - } - - /** MapFunction that filters for records where the rank exceeds a certain threshold. */ - public static class FilterByRank implements FilterFunction> { - - private static final int RANKFILTER = 40; - - /** - * Filters for records of the rank relation where the rank is greater than the given - * threshold. - * - *

Output Format: 0: RANK 1: URL 2: AVG_DURATION - */ - @Override - public boolean filter(Tuple3 value) throws Exception { - return (value.f0 > RANKFILTER); - } - } - - /** - * MapFunction that filters for records of the visits relation where the year (from the date - * string) is equal to a certain value. - */ - public static class FilterVisitsByDate implements FilterFunction> { - - private static final int YEARFILTER = 2007; - - /** - * Filters for records of the visits relation where the year of visit is equal to a - * specified value. The URL of all visit records passing the filter is emitted. - * - *

Output Format: 0: URL 1: DATE - */ - @Override - public boolean filter(Tuple2 value) throws Exception { - // Parse date string with the format YYYY-MM-DD and extract the year - String dateString = value.f1; - int year = Integer.parseInt(dateString.substring(0, 4)); - return (year == YEARFILTER); - } - } - - /** - * CoGroupFunction that realizes an anti-join. If the first input does not provide any pairs, - * all pairs of the second input are emitted. Otherwise, no pair is emitted. - */ - @ForwardedFieldsFirst("*") - public static class AntiJoinVisits - implements CoGroupFunction< - Tuple3, - Tuple1, - Tuple3> { - - /** - * If the visit iterator is empty, all pairs of the rank iterator are emitted. Otherwise, no - * pair is emitted. - * - *

Output Format: 0: RANK 1: URL 2: AVG_DURATION - */ - @Override - public void coGroup( - Iterable> ranks, - Iterable> visits, - Collector> out) { - // Check if there is a entry in the visits relation - if (!visits.iterator().hasNext()) { - for (Tuple3 next : ranks) { - // Emit all rank pairs - out.collect(next); - } - } - } - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static DataSet> getDocumentsDataSet( - ExecutionEnvironment env, ParameterTool params) { - // Create DataSet for documents relation (URL, Doc-Text) - if (params.has("documents")) { - return env.readCsvFile(params.get("documents")) - .fieldDelimiter("|") - .types(String.class, String.class); - } else { - System.out.println("Executing WebLogAnalysis example with default documents data set."); - System.out.println("Use --documents to specify file input."); - return WebLogData.getDocumentDataSet(env); - } - } - - private static DataSet> getRanksDataSet( - ExecutionEnvironment env, ParameterTool params) { - // Create DataSet for ranks relation (Rank, URL, Avg-Visit-Duration) - if (params.has("ranks")) { - return env.readCsvFile(params.get("ranks")) - .fieldDelimiter("|") - .types(Integer.class, String.class, Integer.class); - } else { - System.out.println("Executing WebLogAnalysis example with default ranks data set."); - System.out.println("Use --ranks to specify file input."); - return WebLogData.getRankDataSet(env); - } - } - - private static DataSet> getVisitsDataSet( - ExecutionEnvironment env, ParameterTool params) { - // Create DataSet for visits relation (URL, Date) - if (params.has("visits")) { - return env.readCsvFile(params.get("visits")) - .fieldDelimiter("|") - .includeFields("011000000") - .types(String.class, String.class); - } else { - System.out.println("Executing WebLogAnalysis example with default visits data set."); - System.out.println("Use --visits to specify file input."); - return WebLogData.getVisitDataSet(env); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java deleted file mode 100644 index 11b39ea004967..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/util/WebLogData.java +++ /dev/null @@ -1,721 +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.relational.util; - -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.api.java.tuple.Tuple3; - -import java.util.ArrayList; -import java.util.List; - -/** - * Provides the default data sets used for the Weblog Analysis example program. The default data - * sets are used, if no parameters are given to the program. - */ -public class WebLogData { - - public static final Object[][] DOCUMENTS = { - new Object[] { - "url_0", - "dolor ad amet enim laoreet nostrud veniam aliquip ex nonummy diam dolore tincidunt tation exerci exerci wisi dolor nostrud " - }, - new Object[] { - "url_1", - "wisi minim adipiscing nibh adipiscing ut nibh Lorem Ut nonummy euismod nibh wisi sit consectetuer exerci sed aliquip aliquip dolore aliquam enim dolore veniam aliquam euismod suscipit ad adipiscing exerci aliquip consectetuer euismod aliquip ad exerci ex nibh ex erat exerci laoreet lobortis quis " - }, - new Object[] { - "url_2", - "diam sed convection aliquip amet commodo nonummy sed sed commodo commodo diam commodo adipiscing ad exerci magna exerci tation quis lobortis " - }, - new Object[] { - "url_3", - "exerci suscipit sed lobortis amet lobortis aliquip nibh nostrud ad convection commodo ad nibh sed minim amet ad ea ea " - }, - new Object[] { - "url_4", "sit enim dolor quis laoreet ullamcorper veniam adipiscing ex quis commodo " - }, - new Object[] { - "url_5", - "elit aliquip ea nisl oscillations sit dolor ipsum tincidunt ullamcorper dolore enim adipiscing laoreet elit ea volutpat adipiscing ea nibh nostrud Ut aliquam veniam Lorem laoreet veniam aliquip " - }, - new Object[] { - "url_6", - "consectetuer ad sed suscipit euismod aliquip quis ullamcorper oscillations tation consectetuer tation amet suscipit nibh enim nonummy veniam commodo commodo diam euismod dolor Ut aliquip diam ex ad nonummy ad tincidunt minim exerci consectetuer veniam convection aliquam ut ut Lorem euismod sed ipsum volutpat " - }, - new Object[] { - "url_7", - "Ut volutpat veniam ut consectetuer diam ut aliquam dolor nostrud erat consectetuer adipiscing exerci consectetuer Ut ullamcorper suscipit aliquam sed dolor nisl " - }, - new Object[] { - "url_8", - "suscipit amet wisi nisl veniam lobortis sit Lorem aliquam nostrud aliquam ipsum ut laoreet suscipit Lorem laoreet editors adipiscing ullamcorper veniam erat consectetuer ut lobortis dolore elit sed tincidunt ipsum tation ullamcorper nonummy adipiscing ex ad laoreet ipsum suscipit lobortis lobortis Ut nonummy adipiscing erat volutpat aliquam " - }, - new Object[] { - "url_9", - "nonummy commodo tation editors ut quis sit quis lobortis ea dolore oscillations diam ad dolor lobortis nisl ad veniam ullamcorper quis magna volutpat sit ipsum consectetuer dolore exerci commodo magna erat enim ut suscipit " - }, - new Object[] { - "url_10", - "amet erat magna consectetuer tation tation aliquip nibh aliquam sed adipiscing ut commodo ex erat tincidunt aliquam ipsum Ut Ut sit tincidunt adipiscing suscipit minim sed erat dolor consectetuer Lorem consectetuer Lorem amet nibh diam ea ex enim suscipit wisi dolor nonummy magna enim euismod ullamcorper ut suscipit adipiscing " - }, - new Object[] { - "url_11", - "ex quis exerci tation diam elit nostrud nostrud ut ipsum elit amet diam laoreet amet consectetuer volutpat sed lobortis " - }, - new Object[] { - "url_12", - "elit suscipit sit ullamcorper ut ad erat ut dolor nostrud quis nisl enim erat dolor convection ad minim ut veniam nostrud sed editors adipiscing volutpat Ut aliquip commodo sed euismod adipiscing erat adipiscing dolore nostrud minim sed lobortis ea diam " - }, - new Object[] { - "url_13", - "enim ut quis commodo veniam minim erat lobortis ad diam ex dolor tincidunt exerci ut aliquip tincidunt minim ut magna sed enim wisi veniam oscillations Lorem consectetuer " - }, - new Object[] { - "url_14", - "nibh ipsum ullamcorper volutpat ut wisi dolor quis amet euismod quis ipsum ipsum minim tation volutpat sit exerci volutpat amet nonummy euismod veniam consectetuer sit consectetuer tincidunt nibh aliquam lobortis tation veniam ut ullamcorper wisi magna Ut volutpat consectetuer erat quis dolore ea tation " - }, - new Object[] { - "url_15", - "ad wisi sed enim aliquam oscillations nibh Lorem lobortis veniam nibh laoreet nonummy sed nibh Lorem adipiscing diam magna nostrud magna oscillations ut oscillations elit nostrud diam editors Lorem " - }, - new Object[] { - "url_16", - "nostrud volutpat veniam exerci tincidunt nostrud quis elit ipsum ea nonummy volutpat dolor elit lobortis magna nisl ut ullamcorper magna Lorem exerci nibh nisl magna editors erat aliquam aliquam ullamcorper sit aliquam sit nostrud oscillations consectetuer adipiscing suscipit convection exerci ea ullamcorper ex nisl " - }, - new Object[] { - "url_17", - "ad ex aliquam erat aliquam elit veniam laoreet ut amet amet nostrud ut adipiscing Ut Lorem suscipit ex magna ullamcorper aliquam ullamcorper ullamcorper amet amet commodo aliquam volutpat nonummy nonummy tincidunt amet tation tincidunt volutpat ut veniam nisl erat dolor enim nonummy nostrud adipiscing laoreet adipiscing " - }, - new Object[] { - "url_18", - "lobortis ipsum ex tincidunt tincidunt editors euismod consectetuer ipsum adipiscing lobortis exerci adipiscing nonummy nisl dolore nonummy erat exerci nisl ut dolore wisi volutpat lobortis magna " - }, - new Object[] { - "url_19", "ipsum tation laoreet tation adipiscing wisi nibh diam Ut suscipit ad wisi " - }, - new Object[] { - "url_20", - "diam Lorem enim wisi ad lobortis dolor Ut ipsum amet dolore consectetuer nisl exerci nisl nonummy minim Ut erat oscillations ut Lorem nostrud dolore Ut dolore exerci ad ipsum dolore ex dolore aliquip sed aliquam ex aliquip magna amet ex dolore oscillations aliquip tation magna Ut " - }, - new Object[] { - "url_21", - "lobortis ut amet ex nisl ullamcorper tincidunt ut elit diam quis suscipit ad amet ipsum magna Ut ex tincidunt " - }, - new Object[] { - "url_22", - "amet commodo nisl ad quis lobortis ut commodo sit ut erat exerci lobortis suscipit nibh ut nostrud ut adipiscing commodo commodo quis quis nostrud nisl ipsum nostrud laoreet Lorem nostrud erat nostrud amet consectetuer laoreet oscillations wisi sit magna nibh amet " - }, - new Object[] { - "url_23", - "adipiscing suscipit suscipit aliquip suscipit consectetuer minim magna ea erat nibh sit suscipit sed dolor oscillations nonummy volutpat ut tincidunt " - }, - new Object[] { - "url_24", - "commodo sed tincidunt aliquip aliquip dolore commodo nonummy sed erat ut ex exerci dolore adipiscing tincidunt ex diam amet aliquam " - }, - new Object[] { - "url_25", "consectetuer consectetuer exerci quis ea veniam aliquam laoreet minim ex " - }, - new Object[] { - "url_26", - "dolor exerci euismod minim magna quis erat consectetuer sed ex erat dolore quis ut oscillations ullamcorper Lorem exerci ex nibh ut exerci ullamcorper veniam nibh ut commodo ut Ut nostrud tincidunt tincidunt ad dolore Lorem ea tation enim erat nibh ut ea nonummy sed sed wisi nisl dolore " - }, - new Object[] { - "url_27", - "amet elit ea ea nostrud editors Ut nostrud amet laoreet adipiscing ut nisl nonummy tincidunt ea ipsum ex dolore dolore oscillations sit minim Ut wisi ut laoreet minim elit " - }, - new Object[] { - "url_28", - "wisi exerci volutpat Ut nostrud euismod minim Ut sit euismod ut ea magna consectetuer nisl ad minim tation nisl adipiscing Lorem aliquam quis exerci erat minim aliquip sit Lorem wisi wisi ut " - }, - new Object[] { - "url_29", - "amet sed laoreet amet aliquam minim enim tincidunt Lorem sit aliquip amet suscipit ut laoreet elit suscipit erat ut tincidunt suscipit ipsum sed euismod elit dolore euismod dolore ut dolor nostrud ipsum tincidunt commodo adipiscing aliquam ut wisi dolor dolor suscipit " - }, - new Object[] { - "url_30", - "euismod Lorem ex tincidunt amet enim minim suscipit exerci diam veniam amet nostrud ea ea " - }, - new Object[] { - "url_31", - "ex ipsum sit euismod euismod ullamcorper tincidunt ut wisi ea adipiscing sed diam tation ipsum dolor aliquam veniam nonummy aliquip aliquip Lorem ut minim nisl tation sit exerci ullamcorper Ut dolor euismod aliquam consectetuer ad nonummy commodo exerci " - }, - new Object[] { - "url_32", - "volutpat ipsum lobortis nisl veniam minim adipiscing dolor editors quis nostrud amet nostrud " - }, - new Object[] { - "url_33", - "commodo wisi aliquip ut aliquam sed nostrud ex diam ad nostrud enim ut amet enim ea ad sed tation nostrud suscipit ea magna magna Lorem amet lobortis ut quis nibh aliquam aliquam exerci aliquip lobortis consectetuer enim wisi ea nisl laoreet erat dolore " - }, - new Object[] { - "url_34", - "tincidunt adipiscing enim tation nibh Ut dolore tincidunt tation laoreet suscipit minim aliquam volutpat laoreet suscipit tincidunt nibh ut ut sit nostrud nonummy tincidunt exerci sit ad sed consectetuer minim dolor dolore laoreet nostrud nibh laoreet ea adipiscing exerci dolore ipsum " - }, - new Object[] { - "url_35", - "tation ut erat ut tation dolor Lorem laoreet Lorem elit adipiscing wisi aliquip nostrud elit Ut volutpat ea aliquam aliquip " - }, - new Object[] { - "url_36", - "lobortis enim ullamcorper adipiscing consectetuer aliquip wisi enim minim Ut minim elit elit aliquam exerci ullamcorper amet lobortis adipiscing diam laoreet consectetuer nostrud diam diam amet ut enim ullamcorper aliquip diam ut nostrud diam magna amet nonummy commodo wisi enim ullamcorper suscipit euismod dolore tincidunt magna suscipit elit " - }, - new Object[] { - "url_37", - "elit adipiscing nisl nisl ex aliquip nibh sed ut ad Lorem elit consectetuer ad volutpat lobortis amet veniam ipsum nibh ut consectetuer editors ad aliquam " - }, - new Object[] { - "url_38", - "elit quis nibh adipiscing sit consectetuer ut euismod quis tincidunt quis nisl consectetuer dolor diam suscipit quis dolore Lorem suscipit nonummy sed ex " - }, - new Object[] { - "url_39", - "nisl sit consectetuer elit oscillations enim ipsum enim nostrud adipiscing nostrud editors aliquam " - }, - new Object[] { - "url_40", - "sed wisi dolor diam commodo ullamcorper commodo nostrud ullamcorper laoreet minim dolore suscipit laoreet tation aliquip " - }, - new Object[] { - "url_41", - "ad consectetuer exerci nisl exerci amet enim diam lobortis Lorem ex volutpat volutpat nibh aliquam ut ullamcorper volutpat nostrud ut adipiscing ullamcorper " - }, - new Object[] { - "url_42", - "minim laoreet tation magna veniam ut ea sit ipsum tincidunt Ut amet ex aliquip ex euismod exerci wisi elit editors ad amet veniam ad editors " - }, - new Object[] { - "url_43", - "ut nisl ad ullamcorper nibh Ut editors exerci enim exerci ea laoreet veniam ea amet exerci volutpat amet ad " - }, - new Object[] { - "url_44", - "volutpat tincidunt enim amet sed tincidunt consectetuer ullamcorper nisl Ut adipiscing tation ad ad amet nonummy elit erat nibh Lorem erat elit laoreet consectetuer sed aliquip nostrud " - }, - new Object[] { - "url_45", - "sed aliquam ut ut consectetuer wisi euismod enim erat euismod quis exerci amet tation sit " - }, - new Object[] { - "url_46", - "lobortis oscillations tation aliquam dolore Lorem aliquip tation exerci ullamcorper aliquam aliquip lobortis ex tation dolor ut ut sed suscipit nisl ullamcorper sed editors laoreet aliquip enim dolor veniam tincidunt sed euismod tation " - }, - new Object[] { - "url_47", - "Lorem Lorem ut wisi ad ut tation consectetuer exerci convection tation ullamcorper sed dolore quis aliquam ipsum lobortis commodo nonummy " - }, - new Object[] { - "url_48", - "laoreet minim veniam nisl elit sit amet commodo ex ullamcorper suscipit aliquip laoreet convection Ut ex minim aliquam " - }, - new Object[] { - "url_49", - "lobortis nonummy minim amet sit veniam quis consectetuer tincidunt laoreet quis " - }, - new Object[] { - "url_50", - "lobortis nisl commodo dolor amet nibh editors enim magna minim elit euismod diam laoreet laoreet ad minim sed ut Ut lobortis adipiscing quis sed ut aliquam oscillations exerci tation consectetuer lobortis elit tincidunt consectetuer minim amet dolore quis aliquam Ut exerci sed aliquam quis quis ullamcorper Ut ex tincidunt " - }, - new Object[] { - "url_51", - "nostrud nisl ea erat ut suscipit Ut sit oscillations ullamcorper nonummy magna lobortis dolore editors tincidunt nostrud suscipit ex quis tation ut sit amet nostrud laoreet ex tincidunt " - }, - new Object[] { - "url_52", - "ea tation commodo elit sed ex sed quis enim nisl magna laoreet adipiscing amet sit nostrud consectetuer nibh tincidunt veniam ex veniam euismod exerci sed dolore suscipit nisl tincidunt euismod quis Ut enim euismod dolor diam exerci magna exerci ut exerci nisl " - }, - new Object[] { - "url_53", - "volutpat amet Ut lobortis dolor tation minim nonummy lobortis convection nostrud " - }, - new Object[] { - "url_54", - "ullamcorper commodo Ut amet sit nostrud aliquam ad amet wisi enim nostrud ipsum nisl veniam erat aliquam ex aliquam dolor dolor ut consectetuer euismod exerci elit exerci Ut ea minim enim consectetuer ad consectetuer nonummy convection adipiscing ad ullamcorper lobortis nonummy laoreet nonummy aliquam ullamcorper ad nostrud amet " - }, - new Object[] { - "url_55", - "wisi magna editors amet aliquam diam amet aliquip nisl consectetuer laoreet nonummy suscipit euismod diam enim tation elit ut lobortis quis euismod suscipit nostrud ea ea commodo lobortis dolore Ut nisl nostrud dolor laoreet euismod ea dolore aliquam ut Lorem exerci ex sit " - }, - new Object[] { - "url_56", - "ex dolor veniam wisi laoreet ut exerci diam ad ex ut ut laoreet ut nisl ullamcorper nisl " - }, - new Object[] { - "url_57", - "diam adipiscing Ut ut Lorem amet erat elit erat magna adipiscing euismod elit ullamcorper nostrud aliquam dolor ullamcorper sit tation tation " - }, - new Object[] { - "url_58", - "laoreet convection veniam lobortis dolore ut nonummy commodo erat lobortis veniam nostrud dolore minim commodo ut consectetuer magna erat ea dolore Lorem suscipit ex ipsum exerci sed enim ea tation suscipit enim adipiscing " - }, - new Object[] { - "url_59", - "amet ut ut Ut ad dolor quis ad magna exerci suscipit magna nibh commodo euismod amet euismod wisi diam suscipit dolore Lorem dolor ex amet exerci aliquip ut ut lobortis quis elit minim sed Lorem " - }, - new Object[] { - "url_60", - "ut ut amet ullamcorper amet euismod dolor amet elit exerci adipiscing sed suscipit sed exerci wisi diam veniam wisi suscipit ut quis nibh ullamcorper ex quis magna dolore volutpat editors minim ut sit aliquip oscillations nisl ipsum " - }, - new Object[] { - "url_61", - "nibh nostrud tincidunt lobortis adipiscing adipiscing ullamcorper ullamcorper ipsum nisl ullamcorper aliquip laoreet commodo ut tation wisi diam commodo aliquip commodo suscipit tincidunt volutpat elit enim laoreet ut nostrud ad nonummy ipsum " - }, - new Object[] { - "url_62", - "Ut ut minim enim amet euismod erat elit commodo consectetuer Ut quis dolor ex diam quis wisi tation tincidunt laoreet volutpat " - }, - new Object[] { - "url_63", - "ut erat volutpat euismod amet ea nonummy lobortis ut Ut ea veniam sed veniam nostrud " - }, - new Object[] { - "url_64", - "tation dolor suscipit minim nisl wisi consectetuer aliquip tation Ut commodo ut dolore consectetuer elit wisi nisl ipsum " - }, - new Object[] { - "url_65", "ullamcorper nisl Lorem magna tation veniam aliquam diam amet euismod " - }, - new Object[] { - "url_66", - "euismod aliquam tincidunt Ut volutpat ea lobortis sit ut volutpat ut lobortis ut lobortis ut nisl amet dolor sed ipsum enim ullamcorper diam euismod nostrud wisi erat quis diam nibh Ut dolore sed amet tation enim diam " - }, - new Object[] { - "url_67", - "amet minim minim amet laoreet Lorem aliquam veniam elit volutpat magna adipiscing enim enim euismod laoreet sed ex sed aliquam ad ea ut adipiscing suscipit ex minim dolore minim ea laoreet nisl " - }, - new Object[] { - "url_68", - "aliquam ea volutpat ut wisi tation tation nibh nisl erat laoreet ea volutpat dolor dolor aliquam exerci quis ullamcorper aliquam ut quis suscipit " - }, - new Object[] { - "url_69", - "quis exerci ut aliquip wisi dolore magna nibh consectetuer magna tation ullamcorper lobortis sed amet adipiscing minim suscipit nibh nibh nostrud euismod enim " - }, - new Object[] { - "url_70", - "tation enim consectetuer adipiscing wisi laoreet diam aliquip nostrud elit nostrud aliquip ea minim amet diam dolore " - }, - new Object[] { - "url_71", - "consectetuer tincidunt nibh amet tation nonummy sit tation diam sed diam tation " - }, - new Object[] { - "url_72", - "Lorem ut nostrud nonummy minim quis euismod lobortis nostrud nonummy adipiscing tincidunt consectetuer ut nibh ad suscipit dolor ut elit dolore amet ut quis tation ullamcorper nonummy laoreet ullamcorper aliquam dolore convection dolor tincidunt ut ullamcorper ex dolor suscipit erat oscillations ad " - }, - new Object[] { - "url_73", - "elit Ut commodo ut ullamcorper ullamcorper ut euismod commodo diam aliquip suscipit consectetuer exerci tation nostrud ut wisi exerci sed ut elit sed volutpat Lorem nibh laoreet consectetuer ex Lorem elit aliquam commodo lobortis ad " - }, - new Object[] { - "url_74", - "quis magna laoreet commodo aliquam nisl ullamcorper veniam tation wisi consectetuer commodo consectetuer ad dolore aliquam dolor elit amet sit amet nibh commodo erat veniam aliquip dolore ad magna ad ipsum Ut exerci ea volutpat nisl amet nostrud sit " - }, - new Object[] { - "url_75", - "tincidunt suscipit sit aliquip aliquam adipiscing dolore exerci Ut suscipit ut sit laoreet suscipit wisi sit enim nonummy consectetuer dolore editors " - }, - new Object[] { - "url_76", - "veniam ullamcorper tation sit suscipit dolor suscipit veniam sit Lorem quis sed nostrud ad tincidunt elit adipiscing " - }, - new Object[] { - "url_77", - "volutpat sit amet veniam quis ipsum nibh elit enim commodo magna veniam magna convection " - }, - new Object[] { - "url_78", - "tation dolore minim elit nisl volutpat tation laoreet enim nostrud exerci dolore tincidunt aliquip Lorem ipsum nostrud quis adipiscing ullamcorper erat lobortis tation commodo Ut ipsum commodo magna ad ipsum ut enim " - }, - new Object[] { - "url_79", - "lobortis amet elit Lorem amet nonummy commodo tation ex ea amet Lorem ea nonummy commodo veniam volutpat nibh wisi ad ipsum euismod ea convection nostrud nisl erat veniam Ut aliquip ad aliquip editors wisi magna tation nostrud nonummy adipiscing ullamcorper aliquip " - }, - new Object[] { - "url_80", - "tincidunt nostrud nostrud magna ea euismod ea consectetuer nisl exerci ea dolor nisl commodo ex erat ipsum exerci suscipit ad nisl ea nonummy suscipit adipiscing laoreet sit euismod nibh adipiscing sed minim commodo amet " - }, - new Object[] { - "url_81", - "nostrud erat ut sed editors erat amet magna lobortis diam laoreet dolor amet nibh ut ipsum ipsum amet ut sed ut exerci elit suscipit wisi magna ut veniam nisl commodo enim adipiscing laoreet ad Lorem oscillations " - }, - new Object[] { - "url_82", - "quis commodo nibh nibh volutpat suscipit dolore magna tincidunt nibh ut ad ullamcorper ullamcorper quis enim ad ut tation minim laoreet veniam dolor sed tincidunt exerci exerci nostrud ullamcorper amet ut ut ullamcorper " - }, - new Object[] { - "url_83", - "sit suscipit volutpat elit tation elit sed sed dolor ex ex ipsum euismod laoreet magna lobortis ad " - }, - new Object[] { - "url_84", - "lobortis ipsum euismod enim ea tation veniam tation oscillations aliquip consectetuer euismod ut sed lobortis tation oscillations commodo euismod laoreet suscipit amet elit ullamcorper volutpat aliquam ea enim ullamcorper consectetuer laoreet tation quis ut commodo erat euismod dolor laoreet ullamcorper laoreet " - }, - new Object[] { - "url_85", - "adipiscing sit quis commodo consectetuer quis enim euismod exerci nonummy ea nostrud Ut veniam sit aliquip nisl enim " - }, - new Object[] { - "url_86", - "nostrud dolore veniam veniam wisi aliquip adipiscing diam sed quis ullamcorper " - }, - new Object[] { - "url_87", - "quis Lorem suscipit Ut nibh diam euismod consectetuer lobortis ipsum sed suscipit consectetuer euismod laoreet ut wisi nisl elit quis commodo adipiscing adipiscing suscipit aliquam nisl quis magna ipsum enim ad quis ea magna Lorem nibh ea " - }, - new Object[] { - "url_88", - "euismod commodo sed tincidunt Ut veniam consectetuer quis erat ex ea erat laoreet commodo nibh minim " - }, - new Object[] { - "url_89", - "tation diam editors Ut enim nibh Lorem volutpat quis diam suscipit exerci wisi ad " - }, - new Object[] { - "url_90", - "volutpat editors ea nibh wisi ad amet volutpat nisl ullamcorper nibh volutpat minim ex ut sit veniam Lorem consectetuer quis ad sit suscipit volutpat wisi diam sed tincidunt ipsum minim convection ea diam oscillations quis lobortis " - }, - new Object[] { - "url_91", - "enim minim nonummy ea minim euismod adipiscing editors volutpat magna sit magna ut ipsum ut " - }, - new Object[] { - "url_92", - "nisl Ut commodo amet euismod lobortis ea ea wisi commodo Lorem sit ipsum volutpat nonummy exerci erat elit exerci magna ad erat enim laoreet quis nostrud wisi ut veniam amet ullamcorper lobortis ad suscipit volutpat veniam nostrud nibh quis ipsum dolore consectetuer veniam ipsum aliquip dolore sed laoreet ipsum " - }, - new Object[] { - "url_93", "nonummy aliquam ad lobortis Lorem erat ad tation Lorem exerci ex " - }, - new Object[] { - "url_94", - "nonummy dolore commodo exerci ex quis ut suscipit elit laoreet sit tation magna veniam ea sit nonummy veniam Lorem quis nibh aliquip exerci amet ullamcorper adipiscing erat nisl editors diam commodo ad euismod adipiscing ea suscipit exerci aliquip volutpat tation enim volutpat sit " - }, - new Object[] { - "url_95", - "sit suscipit oscillations ipsum nibh dolor ea dolore ea elit ipsum minim editors magna consectetuer ullamcorper commodo nonummy sit nostrud aliquip sit erat ullamcorper ullamcorper nibh veniam erat quis dolore nonummy " - }, - new Object[] { - "url_96", - "nostrud quis ut volutpat magna ad quis adipiscing Lorem commodo exerci laoreet magna adipiscing erat quis wisi ea ea laoreet enim convection ad dolor nisl amet nibh aliquam adipiscing tincidunt minim diam Lorem commodo adipiscing volutpat " - }, - new Object[] { - "url_97", - "laoreet laoreet suscipit nostrud dolore adipiscing volutpat Ut sed nisl diam ullamcorper ex ut ut dolor amet nostrud euismod dolore veniam veniam enim tation veniam ea minim minim volutpat tincidunt " - }, - new Object[] { - "url_98", - "quis lobortis amet wisi nostrud ipsum aliquam convection tincidunt dolore ullamcorper nibh lobortis volutpat ea nostrud oscillations minim nonummy enim ad lobortis exerci ipsum ullamcorper nibh nonummy diam amet enim veniam ut nostrud " - }, - new Object[] { - "url_99", - "aliquam wisi suscipit commodo diam amet amet magna nisl enim nostrud tation nisl nostrud nibh ut " - } - }; - - public static final Object[][] RANKS = { - new Object[] {30, "url_0", 43}, - new Object[] {82, "url_1", 39}, - new Object[] {56, "url_2", 31}, - new Object[] {96, "url_3", 36}, - new Object[] {31, "url_4", 36}, - new Object[] {29, "url_5", 6}, - new Object[] {33, "url_6", 48}, - new Object[] {66, "url_7", 40}, - new Object[] {28, "url_8", 51}, - new Object[] {9, "url_9", 4}, - new Object[] {49, "url_10", 24}, - new Object[] {26, "url_11", 12}, - new Object[] {39, "url_12", 46}, - new Object[] {84, "url_13", 53}, - new Object[] {29, "url_14", 50}, - new Object[] {21, "url_15", 12}, - new Object[] {69, "url_16", 34}, - new Object[] {11, "url_17", 38}, - new Object[] {96, "url_18", 13}, - new Object[] {56, "url_19", 48}, - new Object[] {18, "url_20", 36}, - new Object[] {31, "url_21", 21}, - new Object[] {29, "url_22", 11}, - new Object[] {71, "url_23", 30}, - new Object[] {85, "url_24", 48}, - new Object[] {19, "url_25", 45}, - new Object[] {69, "url_26", 9}, - new Object[] {20, "url_27", 51}, - new Object[] {33, "url_28", 46}, - new Object[] {75, "url_29", 38}, - new Object[] {96, "url_30", 51}, - new Object[] {73, "url_31", 40}, - new Object[] {67, "url_32", 16}, - new Object[] {24, "url_33", 24}, - new Object[] {27, "url_34", 35}, - new Object[] {33, "url_35", 35}, - new Object[] {7, "url_36", 22}, - new Object[] {83, "url_37", 41}, - new Object[] {23, "url_38", 49}, - new Object[] {41, "url_39", 33}, - new Object[] {66, "url_40", 38}, - new Object[] {4, "url_41", 52}, - new Object[] {34, "url_42", 4}, - new Object[] {28, "url_43", 12}, - new Object[] {14, "url_44", 14}, - new Object[] {41, "url_45", 11}, - new Object[] {48, "url_46", 37}, - new Object[] {75, "url_47", 41}, - new Object[] {78, "url_48", 3}, - new Object[] {63, "url_49", 28} - }; - - public static final Object[][] VISITS = { - new Object[] {"url_2", "2003-12-17"}, - new Object[] {"url_9", "2008-11-11"}, - new Object[] {"url_14", "2003-11-5"}, - new Object[] {"url_46", "2009-2-16"}, - new Object[] {"url_14", "2004-11-9"}, - new Object[] {"url_36", "2001-3-9"}, - new Object[] {"url_35", "2006-8-13"}, - new Object[] {"url_22", "2008-1-18"}, - new Object[] {"url_36", "2002-3-9"}, - new Object[] {"url_13", "2007-7-17"}, - new Object[] {"url_23", "2009-6-16"}, - new Object[] {"url_16", "2000-7-15"}, - new Object[] {"url_41", "2002-5-10"}, - new Object[] {"url_6", "2004-11-9"}, - new Object[] {"url_5", "2003-6-7"}, - new Object[] {"url_22", "2002-11-5"}, - new Object[] {"url_11", "2007-7-21"}, - new Object[] {"url_38", "2009-12-2"}, - new Object[] {"url_6", "2004-11-2"}, - new Object[] {"url_46", "2000-6-4"}, - new Object[] {"url_34", "2003-9-2"}, - new Object[] {"url_31", "2008-2-24"}, - new Object[] {"url_0", "2003-2-2"}, - new Object[] {"url_47", "2003-7-8"}, - new Object[] {"url_49", "2009-9-13"}, - new Object[] {"url_11", "2003-4-2"}, - new Object[] {"url_20", "2000-6-18"}, - new Object[] {"url_38", "2000-2-22"}, - new Object[] {"url_44", "2009-2-17"}, - new Object[] {"url_26", "2000-6-21"}, - new Object[] {"url_13", "2000-11-25"}, - new Object[] {"url_47", "2005-4-19"}, - new Object[] {"url_46", "2008-1-7"}, - new Object[] {"url_33", "2004-12-24"}, - new Object[] {"url_32", "2009-2-8"}, - new Object[] {"url_26", "2000-9-21"}, - new Object[] {"url_9", "2002-8-18"}, - new Object[] {"url_38", "2002-11-27"}, - new Object[] {"url_37", "2008-2-26"}, - new Object[] {"url_1", "2007-3-22"}, - new Object[] {"url_37", "2002-3-20"}, - new Object[] {"url_27", "2008-11-12"}, - new Object[] {"url_30", "2000-12-16"}, - new Object[] {"url_48", "2000-12-17"}, - new Object[] {"url_46", "2008-4-16"}, - new Object[] {"url_29", "2006-3-9"}, - new Object[] {"url_0", "2007-7-26"}, - new Object[] {"url_46", "2009-12-15"}, - new Object[] {"url_34", "2002-2-13"}, - new Object[] {"url_24", "2009-3-1"}, - new Object[] {"url_43", "2007-11-4"}, - new Object[] {"url_3", "2004-2-16"}, - new Object[] {"url_26", "2000-10-26"}, - new Object[] {"url_42", "2004-7-14"}, - new Object[] {"url_13", "2004-9-10"}, - new Object[] {"url_21", "2000-2-21"}, - new Object[] {"url_9", "2006-6-5"}, - new Object[] {"url_46", "2001-12-17"}, - new Object[] {"url_24", "2006-12-8"}, - new Object[] {"url_25", "2006-9-2"}, - new Object[] {"url_37", "2002-6-26"}, - new Object[] {"url_18", "2006-6-2"}, - new Object[] {"url_46", "2003-5-24"}, - new Object[] {"url_32", "2000-10-17"}, - new Object[] {"url_45", "2002-1-12"}, - new Object[] {"url_12", "2005-12-13"}, - new Object[] {"url_49", "2009-3-9"}, - new Object[] {"url_31", "2001-9-19"}, - new Object[] {"url_22", "2002-7-9"}, - new Object[] {"url_27", "2005-2-3"}, - new Object[] {"url_43", "2008-7-15"}, - new Object[] {"url_20", "2000-3-23"}, - new Object[] {"url_25", "2002-5-8"}, - new Object[] {"url_41", "2004-4-27"}, - new Object[] {"url_17", "2008-7-17"}, - new Object[] {"url_26", "2009-12-16"}, - new Object[] {"url_34", "2006-2-10"}, - new Object[] {"url_8", "2009-4-14"}, - new Object[] {"url_16", "2000-2-24"}, - new Object[] {"url_2", "2009-2-10"}, - new Object[] {"url_35", "2003-2-24"}, - new Object[] {"url_34", "2008-3-16"}, - new Object[] {"url_27", "2005-1-5"}, - new Object[] {"url_8", "2008-12-10"}, - new Object[] {"url_38", "2009-2-11"}, - new Object[] {"url_38", "2006-11-3"}, - new Object[] {"url_47", "2003-2-13"}, - new Object[] {"url_8", "2008-11-17"}, - new Object[] {"url_26", "2009-5-11"}, - new Object[] {"url_12", "2007-11-26"}, - new Object[] {"url_10", "2003-1-13"}, - new Object[] {"url_8", "2005-9-23"}, - new Object[] {"url_42", "2001-4-5"}, - new Object[] {"url_30", "2009-12-10"}, - new Object[] {"url_2", "2003-1-3"}, - new Object[] {"url_2", "2009-2-19"}, - new Object[] {"url_7", "2000-6-25"}, - new Object[] {"url_15", "2004-9-26"}, - new Object[] {"url_25", "2009-10-5"}, - new Object[] {"url_23", "2009-8-9"}, - new Object[] {"url_27", "2004-4-3"}, - new Object[] {"url_37", "2008-6-9"}, - new Object[] {"url_9", "2002-5-25"}, - new Object[] {"url_43", "2009-5-18"}, - new Object[] {"url_21", "2008-4-19"}, - new Object[] {"url_12", "2001-12-25"}, - new Object[] {"url_16", "2006-9-25"}, - new Object[] {"url_27", "2002-1-2"}, - new Object[] {"url_2", "2009-1-21"}, - new Object[] {"url_31", "2009-3-20"}, - new Object[] {"url_42", "2002-3-1"}, - new Object[] {"url_31", "2001-11-26"}, - new Object[] {"url_20", "2003-5-15"}, - new Object[] {"url_32", "2004-1-22"}, - new Object[] {"url_28", "2008-9-16"}, - new Object[] {"url_27", "2006-7-3"}, - new Object[] {"url_11", "2008-12-26"}, - new Object[] {"url_15", "2004-8-16"}, - new Object[] {"url_34", "2002-10-5"}, - new Object[] {"url_44", "2000-2-15"}, - new Object[] {"url_9", "2000-10-23"}, - new Object[] {"url_45", "2005-4-24"}, - new Object[] {"url_0", "2006-8-7"}, - new Object[] {"url_48", "2003-8-7"}, - new Object[] {"url_8", "2007-12-13"}, - new Object[] {"url_42", "2003-8-2"}, - new Object[] {"url_25", "2008-3-5"}, - new Object[] {"url_3", "2007-3-9"}, - new Object[] {"url_49", "2003-10-7"}, - new Object[] {"url_18", "2007-12-6"}, - new Object[] {"url_3", "2006-7-5"}, - new Object[] {"url_27", "2000-9-14"}, - new Object[] {"url_42", "2002-10-20"}, - new Object[] {"url_44", "2007-1-13"}, - new Object[] {"url_6", "2003-1-21"}, - new Object[] {"url_40", "2009-10-20"}, - new Object[] {"url_28", "2009-6-17"}, - new Object[] {"url_22", "2000-2-17"}, - new Object[] {"url_3", "2005-1-15"}, - new Object[] {"url_9", "2008-12-9"}, - new Object[] {"url_9", "2005-2-19"}, - new Object[] {"url_28", "2000-4-22"}, - new Object[] {"url_44", "2001-9-9"}, - new Object[] {"url_43", "2008-6-21"}, - new Object[] {"url_39", "2008-5-9"}, - new Object[] {"url_15", "2006-9-15"}, - new Object[] {"url_23", "2001-12-18"}, - new Object[] {"url_14", "2002-5-23"}, - new Object[] {"url_11", "2007-7-11"}, - new Object[] {"url_34", "2000-12-8"}, - new Object[] {"url_47", "2005-7-3"}, - new Object[] {"url_38", "2004-3-26"}, - new Object[] {"url_19", "2003-9-14"}, - new Object[] {"url_24", "2007-7-16"}, - new Object[] {"url_40", "2008-8-21"}, - new Object[] {"url_17", "2007-12-4"}, - new Object[] {"url_25", "2006-6-24"}, - new Object[] {"url_2", "2000-10-8"}, - new Object[] {"url_12", "2008-6-10"}, - new Object[] {"url_11", "2004-11-24"}, - new Object[] {"url_13", "2005-11-3"}, - new Object[] {"url_43", "2005-1-2"}, - new Object[] {"url_14", "2008-6-12"}, - new Object[] {"url_43", "2001-8-27"}, - new Object[] {"url_45", "2000-3-3"}, - new Object[] {"url_0", "2006-9-27"}, - new Object[] {"url_22", "2007-12-18"}, - new Object[] {"url_25", "2006-4-4"}, - new Object[] {"url_32", "2001-6-25"}, - new Object[] {"url_6", "2007-6-9"}, - new Object[] {"url_8", "2009-10-3"}, - new Object[] {"url_15", "2003-2-23"}, - new Object[] {"url_37", "2000-5-6"}, - new Object[] {"url_27", "2004-3-21"}, - new Object[] {"url_17", "2005-6-20"}, - new Object[] {"url_2", "2004-2-27"}, - new Object[] {"url_36", "2005-3-16"}, - new Object[] {"url_1", "2009-12-3"}, - new Object[] {"url_9", "2004-4-27"}, - new Object[] {"url_18", "2009-5-26"}, - new Object[] {"url_31", "2000-9-21"}, - new Object[] {"url_12", "2008-9-25"}, - new Object[] {"url_2", "2004-2-16"}, - new Object[] {"url_28", "2008-11-12"}, - new Object[] {"url_28", "2001-6-26"}, - new Object[] {"url_12", "2006-3-15"}, - new Object[] {"url_0", "2009-3-1"}, - new Object[] {"url_36", "2006-10-13"}, - new Object[] {"url_15", "2004-11-5"}, - new Object[] {"url_32", "2008-2-11"}, - new Object[] {"url_19", "2009-8-3"}, - new Object[] {"url_2", "2006-8-6"}, - new Object[] {"url_11", "2009-10-13"}, - new Object[] {"url_21", "2002-9-14"}, - new Object[] {"url_18", "2000-11-2"}, - new Object[] {"url_35", "2006-5-15"}, - new Object[] {"url_11", "2006-2-18"}, - new Object[] {"url_0", "2001-4-25"}, - new Object[] {"url_14", "2009-4-8"}, - new Object[] {"url_16", "2009-4-7"} - }; - - public static DataSet> getDocumentDataSet(ExecutionEnvironment env) { - - List> data = new ArrayList>(100); - for (Object[] document : DOCUMENTS) { - data.add(new Tuple2((String) document[0], (String) document[1])); - } - - return env.fromCollection(data); - } - - public static DataSet> getRankDataSet( - ExecutionEnvironment env) { - - List> data = - new ArrayList>(100); - for (Object[] rank : RANKS) { - data.add( - new Tuple3( - (Integer) rank[0], (String) rank[1], (Integer) rank[2])); - } - return env.fromCollection(data); - } - - public static DataSet> getVisitDataSet(ExecutionEnvironment env) { - - List> data = new ArrayList>(100); - - for (Object[] visit : VISITS) { - data.add(new Tuple2((String) visit[0], (String) visit[1])); - } - return env.fromCollection(data); - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java deleted file mode 100644 index fee27b98e957a..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/util/WebLogDataGenerator.java +++ /dev/null @@ -1,226 +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.relational.util; - -import org.apache.flink.examples.java.relational.WebLogAnalysis; - -import java.io.BufferedWriter; -import java.io.FileWriter; -import java.io.IOException; -import java.util.Calendar; -import java.util.Random; - -/** Data generator for the {@link WebLogAnalysis} example program. */ -public class WebLogDataGenerator { - - /** - * Main method to generate data for the {@link WebLogAnalysis} example program. - * - *

The generator creates to files: - * - *

    - *
  • {tmp.dir}/documents for the web documents - *
  • {tmp.dir}/ranks for the ranks of the web documents - *
  • {tmp.dir}/visits for the logged visits of web documents - *
- * - * @param args - *
    - *
  1. Int: Number of web documents - *
  2. Int: Number of visits - *
- */ - public static void main(String[] args) { - - // parse parameters - if (args.length < 2) { - System.out.println("WebLogDataGenerator "); - System.exit(1); - } - - int noDocs = Integer.parseInt(args[0]); - int noVisits = Integer.parseInt(args[1]); - - String[] filterKWs = {"editors", "oscillations", "convection"}; - - String[] words = { - "Lorem", - "ipsum", - "dolor", - "sit", - "amet", - "consectetuer", - "adipiscing", - "elit", - "sed", - "diam", - "nonummy", - "nibh", - "euismod", - "tincidunt", - "ut", - "laoreet", - "dolore", - "magna", - "aliquam", - "erat", - "volutpat", - "Ut", - "wisi", - "enim", - "ad", - "minim", - "veniam", - "quis", - "nostrud", - "exerci", - "tation", - "ullamcorper", - "suscipit", - "lobortis", - "nisl", - "ut", - "aliquip", - "ex", - "ea", - "commodo" - }; - - final String outPath = System.getProperty("java.io.tmpdir"); - - System.out.println("Generating documents files..."); - genDocs(noDocs, filterKWs, words, outPath + "/documents"); - System.out.println("Generating ranks files..."); - genRanks(noDocs, outPath + "/ranks"); - System.out.println("Generating visits files..."); - genVisits(noVisits, noDocs, outPath + "/visits"); - - System.out.println("Done!"); - } - - /** - * Generates the files for the documents relation. The entries apply the following format:
- * URL | Content - * - * @param noDocs Number of entries for the documents relation - * @param filterKeyWords A list of keywords that should be contained - * @param words A list of words to fill the entries - * @param path Output path for the documents relation - */ - private static void genDocs(int noDocs, String[] filterKeyWords, String[] words, String path) { - - Random rand = new Random(Calendar.getInstance().getTimeInMillis()); - - try (BufferedWriter fw = new BufferedWriter(new FileWriter(path))) { - for (int i = 0; i < noDocs; i++) { - - int wordsInDoc = rand.nextInt(40) + 10; - // URL - StringBuilder doc = new StringBuilder("url_" + i + "|"); - for (int j = 0; j < wordsInDoc; j++) { - if (rand.nextDouble() > 0.9) { - // Approx. every 10th word is a keyword - doc.append(filterKeyWords[rand.nextInt(filterKeyWords.length)] + " "); - } else { - // Fills up the docs file(s) with random words - doc.append(words[rand.nextInt(words.length)] + " "); - } - } - doc.append("|\n"); - - fw.write(doc.toString()); - } - } catch (IOException e) { - e.printStackTrace(); - } - } - - /** - * Generates the files for the ranks relation. The ranks entries apply the following format: - *
- * Rank | URL | Average Duration |\n - * - * @param noDocs Number of entries in the documents relation - * @param path Output path for the ranks relation - */ - private static void genRanks(int noDocs, String path) { - - Random rand = new Random(Calendar.getInstance().getTimeInMillis()); - - try (BufferedWriter fw = new BufferedWriter(new FileWriter(path))) { - for (int i = 0; i < noDocs; i++) { - // Rank - StringBuilder rank = new StringBuilder(rand.nextInt(100) + "|"); - // URL - rank.append("url_" + i + "|"); - // Average duration - rank.append(rand.nextInt(10) + rand.nextInt(50) + "|\n"); - - fw.write(rank.toString()); - } - } catch (IOException e) { - e.printStackTrace(); - } - } - - /** - * Generates the files for the visits relation. The visits entries apply the following format: - *
- * IP Address | URL | Date (YYYY-MM-DD) | Misc. Data (e.g. User-Agent) |\n - * - * @param noVisits Number of entries for the visits relation - * @param noDocs Number of entries in the documents relation - * @param path Output path for the visits relation - */ - private static void genVisits(int noVisits, int noDocs, String path) { - - Random rand = new Random(Calendar.getInstance().getTimeInMillis()); - - try (BufferedWriter fw = new BufferedWriter(new FileWriter(path))) { - for (int i = 0; i < noVisits; i++) { - - int year = 2000 + rand.nextInt(10); // yearFilter 3 - int month = rand.nextInt(12) + 1; // month between 1 and 12 - int day = rand.nextInt(27) + 1; // day between 1 and 28 - - // IP address - StringBuilder visit = - new StringBuilder( - rand.nextInt(256) - + "." - + rand.nextInt(256) - + "." - + rand.nextInt(256) - + "." - + rand.nextInt(256) - + "|"); - // URL - visit.append("url_" + rand.nextInt(noDocs) + "|"); - // Date (format: YYYY-MM-DD) - visit.append(year + "-" + month + "-" + day + "|"); - // Miscellaneous data, e.g. User-Agent - visit.append("0.12|Mozilla Firefox 3.1|de|de|Nothing special|124|\n"); - - fw.write(visit.toString()); - } - } catch (IOException e) { - e.printStackTrace(); - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/util/DataSetDeprecationInfo.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/util/DataSetDeprecationInfo.java deleted file mode 100644 index 8a5a97f43481b..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/util/DataSetDeprecationInfo.java +++ /dev/null @@ -1,29 +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.util; - -/** The info about the deprecation of DataSet API. */ -public class DataSetDeprecationInfo { - - public static final String DATASET_DEPRECATION_INFO = - "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." - + " See Also: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741"; -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java deleted file mode 100644 index 7278f29f73ab5..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java +++ /dev/null @@ -1,138 +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.wordcount; - -import org.apache.flink.api.common.functions.FlatMapFunction; -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.api.java.utils.MultipleParameterTool; -import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.util.Collector; -import org.apache.flink.util.Preconditions; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * Implements the "WordCount" program that computes a simple word occurrence histogram over text - * files. - * - *

The input is a plain text file with lines separated by newline characters. - * - *

Usage: WordCount --input <path> --output <path>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}. - * - *

This example shows how to: - * - *

    - *
  • write a simple Flink program. - *
  • use Tuple data types. - *
  • write and use user-defined functions. - *
- * - *

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 WordCount { - - private static final Logger LOGGER = LoggerFactory.getLogger(WordCount.class); - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final MultipleParameterTool params = MultipleParameterTool.fromArgs(args); - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // get input data - DataSet text = null; - if (params.has("input")) { - // union all the inputs from text files - for (String input : params.getMultiParameterRequired("input")) { - if (text == null) { - text = env.readTextFile(input); - } else { - text = text.union(env.readTextFile(input)); - } - } - Preconditions.checkNotNull(text, "Input DataSet should not be null."); - } else { - // get default test text data - System.out.println("Executing WordCount example with default input data set."); - System.out.println("Use --input to specify file input."); - text = WordCountData.getDefaultTextLineDataSet(env); - } - - DataSet> 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) - .sum(1); - - // emit result - if (params.has("output")) { - counts.writeAsCsv(params.get("output"), "\n", " "); - // execute program - env.execute("WordCount Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - counts.print(); - } - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into multiple pairs in the - * form of "(word,1)" ({@code Tuple2}). - */ - public static final class Tokenizer - implements FlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Tuple2<>(token, 1)); - } - } - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCountPojo.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCountPojo.java deleted file mode 100644 index 1d569385beb8d..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCountPojo.java +++ /dev/null @@ -1,160 +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.wordcount; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.util.Collector; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO; - -/** - * This example shows an implementation of WordCount without using the Tuple2 type, but a custom - * class. - * - *

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 WordCountPojo { - - private static final Logger LOGGER = LoggerFactory.getLogger(WordCountPojo.class); - - /** - * This is the POJO (Plain Old Java Object) that is being used for all the operations. As long - * as all fields are public or have a getter/setter, the system can handle them - */ - public static class Word { - - // fields - private String word; - private int frequency; - - // constructors - public Word() {} - - public Word(String word, int i) { - this.word = word; - this.frequency = i; - } - - // getters setters - public String getWord() { - return word; - } - - public void setWord(String word) { - this.word = word; - } - - public int getFrequency() { - return frequency; - } - - public void setFrequency(int frequency) { - this.frequency = frequency; - } - - @Override - public String toString() { - return "Word=" + word + " freq=" + frequency; - } - } - - public static void main(String[] args) throws Exception { - - LOGGER.warn(DATASET_DEPRECATION_INFO); - - final ParameterTool params = ParameterTool.fromArgs(args); - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // make parameters available in the web interface - env.getConfig().setGlobalJobParameters(params); - - // get input data - DataSet text; - if (params.has("input")) { - // read the text file from given input path - text = env.readTextFile(params.get("input")); - } else { - // get default test text data - System.out.println("Executing WordCount example with default input data set."); - System.out.println("Use --input to specify file input."); - text = WordCountData.getDefaultTextLineDataSet(env); - } - - DataSet counts = - // split up the lines into Word objects (with frequency = 1) - text.flatMap(new Tokenizer()) - // group by the field word and sum up the frequency - .groupBy("word") - .reduce( - new ReduceFunction() { - @Override - public Word reduce(Word value1, Word value2) throws Exception { - return new Word( - value1.word, value1.frequency + value2.frequency); - } - }); - - if (params.has("output")) { - counts.writeAsText(params.get("output"), WriteMode.OVERWRITE); - // execute program - env.execute("WordCount-Pojo Example"); - } else { - System.out.println("Printing result to stdout. Use --output to specify output path."); - counts.print(); - } - } - - // ************************************************************************* - // USER FUNCTIONS - // ************************************************************************* - - /** - * Implements the string tokenizer that splits sentences into words as a user-defined - * FlatMapFunction. The function takes a line (String) and splits it into multiple Word objects. - */ - public static final class Tokenizer implements FlatMapFunction { - - @Override - public void flatMap(String value, Collector out) { - // normalize and split the line - String[] tokens = value.toLowerCase().split("\\W+"); - - // emit the pairs - for (String token : tokens) { - if (token.length() > 0) { - out.collect(new Word(token, 1)); - } - } - } - } -} diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java deleted file mode 100644 index 978256c4b8a43..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/util/WordCountData.java +++ /dev/null @@ -1,72 +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.wordcount.util; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; - -/** - * Provides the default data sets used for the WordCount example program. The default data sets are - * used, if no parameters are given to the program. - */ -public class WordCountData { - - public static final String[] WORDS = - new String[] { - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,", - "And by opposing end them?--To die,--to sleep,--", - "No more; and by a sleep to say we end", - "The heartache, and the thousand natural shocks", - "That flesh is heir to,--'tis a consummation", - "Devoutly to be wish'd. To die,--to sleep;--", - "To sleep! perchance to dream:--ay, there's the rub;", - "For in that sleep of death what dreams may come,", - "When we have shuffled off this mortal coil,", - "Must give us pause: there's the respect", - "That makes calamity of so long life;", - "For who would bear the whips and scorns of time,", - "The oppressor's wrong, the proud man's contumely,", - "The pangs of despis'd love, the law's delay,", - "The insolence of office, and the spurns", - "That patient merit of the unworthy takes,", - "When he himself might his quietus make", - "With a bare bodkin? who would these fardels bear,", - "To grunt and sweat under a weary life,", - "But that the dread of something after death,--", - "The undiscover'd country, from whose bourn", - "No traveller returns,--puzzles the will,", - "And makes us rather bear those ills we have", - "Than fly to others that we know not of?", - "Thus conscience does make cowards of us all;", - "And thus the native hue of resolution", - "Is sicklied o'er with the pale cast of thought;", - "And enterprises of great pith and moment,", - "With this regard, their currents turn awry,", - "And lose the name of action.--Soft you now!", - "The fair Ophelia!--Nymph, in thy orisons", - "Be all my sins remember'd." - }; - - public static DataSet getDefaultTextLineDataSet(ExecutionEnvironment env) { - return env.fromElements(WORDS); - } -} diff --git a/flink-examples/flink-examples-batch/src/main/resources/log4j2.properties b/flink-examples/flink-examples-batch/src/main/resources/log4j2.properties deleted file mode 100644 index 9206863eda178..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/resources/log4j2.properties +++ /dev/null @@ -1,25 +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. -################################################################################ - -rootLogger.level = INFO -rootLogger.appenderRef.console.ref = ConsoleAppender - -appender.console.name = ConsoleAppender -appender.console.type = CONSOLE -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/flink-examples/flink-examples-batch/src/main/resources/logback.xml b/flink-examples/flink-examples-batch/src/main/resources/logback.xml deleted file mode 100644 index 95f2d0419393b..0000000000000 --- a/flink-examples/flink-examples-batch/src/main/resources/logback.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n - - - - - - - \ No newline at end of file diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java index 2d96426805047..c587dc6d0f6ec 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.connector.datagen.source.DataGeneratorSource; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; @@ -29,6 +28,7 @@ import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.util.ParameterTool; import java.util.Collections; import java.util.concurrent.TimeUnit; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java index c437c40ee9e96..fd3cde36d2ad8 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java @@ -24,13 +24,13 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.connector.datagen.source.DataGeneratorSource; import org.apache.flink.connector.datagen.source.GeneratorFunction; import org.apache.flink.connector.file.sink.FileSink; import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.ParameterTool; import org.apache.flink.util.Preconditions; import jcuda.Pointer; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java index abd44392a604d..699d54b5dd086 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.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.tuple.Tuple3; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.file.sink.FileSink; import org.apache.flink.core.fs.Path; @@ -36,6 +35,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.util.ParameterTool; import java.time.Duration; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java index 6e33f5d9ffc5a..cfc32338b5910 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java @@ -26,7 +26,6 @@ import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.file.sink.FileSink; import org.apache.flink.connector.file.src.FileSource; @@ -42,6 +41,7 @@ import org.apache.flink.streaming.examples.wordcount.util.WordCountData; import org.apache.flink.util.Collector; import org.apache.flink.util.OutputTag; +import org.apache.flink.util.ParameterTool; import java.time.Duration; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java index 584183a3ba9d9..213edaff66aa7 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java @@ -20,10 +20,10 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; +import org.apache.flink.util.ParameterTool; import java.time.Duration; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java index 7cde1762f0482..d1c8dadcacffb 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.connector.datagen.source.DataGeneratorSource; import org.apache.flink.connector.datagen.source.GeneratorFunction; import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; @@ -31,6 +30,7 @@ import org.apache.flink.streaming.examples.statemachine.event.Event; import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorFunction; import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializationSchema; +import org.apache.flink.util.ParameterTool; /** * Job to generate input events that are written to Kafka, for the {@link StateMachineExample} job. diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java index 940b836d0bf03..28c9d048f0821 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java @@ -26,7 +26,6 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; @@ -47,6 +46,7 @@ import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorFunction; import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializationSchema; import org.apache.flink.util.Collector; +import org.apache.flink.util.ParameterTool; import java.time.Duration; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java index 5708a7ae90e11..490c773c2f1c1 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.datagen.source.DataGeneratorSource; import org.apache.flink.connector.datagen.source.GeneratorFunction; @@ -32,6 +31,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; +import org.apache.flink.util.ParameterTool; import java.time.Duration; import java.util.ArrayList; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java index ddf11113e2047..7d9e831402f83 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/util/CLI.java @@ -19,9 +19,9 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.java.utils.MultipleParameterTool; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.core.fs.Path; +import org.apache.flink.util.MultipleParameterTool; import org.apache.flink.util.TimeUtils; import java.time.Duration; diff --git a/flink-examples/flink-examples-streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java b/flink-examples/flink-examples-streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java index d55b9bb1b4394..ac5d6b010b987 100644 --- a/flink-examples/flink-examples-streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java +++ b/flink-examples/flink-examples-streaming/src/test/java/org/apache/flink/streaming/test/StreamingExamplesITCase.java @@ -25,8 +25,11 @@ import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; +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.api.java.tuple.Tuple3; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.test.examples.join.WindowJoinData; @@ -37,6 +40,7 @@ import org.junit.Test; import java.io.File; +import java.nio.file.Files; import static org.apache.flink.test.util.TestBaseUtils.checkLinesAgainstRegexp; import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; @@ -47,7 +51,7 @@ public class StreamingExamplesITCase extends AbstractTestBaseJUnit4 { @Test public void testWindowJoin() throws Exception { - final String resultPath = File.createTempFile("result-path", "dir").toURI().toString(); + final String resultPath = Files.createTempDirectory("result-path").toUri().toString(); final class Parser implements MapFunction> { @@ -73,7 +77,12 @@ public Tuple2 map(String value) throws Exception { .map(new Parser()); org.apache.flink.streaming.examples.join.WindowJoin.runWindowJoin(grades, salaries, 100) - .writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE); + .sinkTo( + FileSink.forRowFormat( + new Path(resultPath), + new SimpleStringEncoder< + Tuple3>()) + .build()); env.execute(); diff --git a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/connectors/ChangelogSocketExample.java b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/connectors/ChangelogSocketExample.java index b0f894a1a8b7c..0a6ef049516ed 100644 --- a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/connectors/ChangelogSocketExample.java +++ b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/connectors/ChangelogSocketExample.java @@ -19,13 +19,13 @@ package org.apache.flink.table.examples.java.connectors; import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.util.ParameterTool; /** * Example for implementing a custom {@link DynamicTableSource} and a {@link DecodingFormat}. diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 9100fd53b70e5..15fcffebcd1d0 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -36,7 +36,6 @@ under the License. - flink-examples-batch flink-examples-streaming flink-examples-table flink-examples-build-helper diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java index c2b00443a5695..4a83c6997c1cc 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java @@ -23,7 +23,7 @@ import org.apache.flink.formats.avro.testjar.AvroExternalJarProgram; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; -import org.apache.flink.test.util.TestEnvironment; +import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.JarUtils; import org.junit.jupiter.api.AfterAll; @@ -58,7 +58,7 @@ static void setUp() throws Exception { @AfterAll static void tearDown() { - TestEnvironment.unsetAsContext(); + TestStreamEnvironment.unsetAsContext(); MINI_CLUSTER.closeAsync(); } @@ -72,7 +72,7 @@ void testExternalProgram() throws Exception { jarFile = "target/".concat(jarFile); } - TestEnvironment.setAsContext( + TestStreamEnvironment.setAsContext( MINI_CLUSTER, PARALLELISM, Collections.singleton(new Path(jarFile)), diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroInputFormatTypeExtractionTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroInputFormatTypeExtractionTest.java index 59f5bb8a29ef4..97a03b161e226 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroInputFormatTypeExtractionTest.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroInputFormatTypeExtractionTest.java @@ -20,11 +20,11 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Test; @@ -43,8 +43,8 @@ void testTypeExtraction() { TypeInformation typeInfoDirect = TypeExtractor.getInputFormatTypes(format); - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet input = env.createInput(format); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStream input = env.createInput(format); TypeInformation typeInfoDataSet = input.getType(); assertThat(typeInfoDirect).isInstanceOf(PojoTypeInfo.class); diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatITCase.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatITCase.java index cd0015dc1fd71..1347366cf8551 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatITCase.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatITCase.java @@ -19,13 +19,18 @@ package org.apache.flink.formats.avro; 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.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.core.fs.Path; import org.apache.flink.formats.avro.AvroOutputFormat.Codec; import org.apache.flink.formats.avro.generated.Colors; import org.apache.flink.formats.avro.generated.Fixed2; import org.apache.flink.formats.avro.generated.User; +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.legacy.io.TextInputFormat; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.apache.avro.file.DataFileReader; @@ -70,24 +75,33 @@ protected void preSubmit() throws Exception { @Override protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = - env.readCsvFile(inputPath) - .fieldDelimiter("|") - .types(String.class, Integer.class, String.class); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStream> input = + env.createInput(new TextInputFormat(new Path(inputPath))) + .map( + x -> { + String[] splits = x.split("\\|"); + return Tuple3.of( + splits[0], Integer.valueOf(splits[1]), splits[2]); + }) + .returns( + TypeInformation.of( + new TypeHint>() {})); // output the data with AvroOutputFormat for specific user type - DataSet specificUser = input.map(new ConvertToUser()); - AvroOutputFormat avroOutputFormat = new AvroOutputFormat<>(User.class); + DataStream specificUser = input.map(new ConvertToUser()); + AvroOutputFormat avroOutputFormat = + new AvroOutputFormat<>(new Path(outputPath1), User.class); avroOutputFormat.setCodec(Codec.SNAPPY); // FLINK-4771: use a codec avroOutputFormat.setSchema( User.SCHEMA$); // FLINK-3304: Ensure the OF is properly serializing the schema - specificUser.write(avroOutputFormat, outputPath1); + specificUser.addSink(new OutputFormatSinkFunction<>(avroOutputFormat)); // output the data with AvroOutputFormat for reflect user type - DataSet reflectiveUser = specificUser.map(new ConvertToReflective()); - reflectiveUser.write(new AvroOutputFormat<>(ReflectiveUser.class), outputPath2); + DataStream reflectiveUser = specificUser.map(new ConvertToReflective()); + reflectiveUser.addSink( + new OutputFormatSinkFunction<>( + new AvroOutputFormat<>(new Path(outputPath2), ReflectiveUser.class))); env.execute(); } diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java index 5e61295cf699b..09163764e5bd3 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java @@ -20,12 +20,13 @@ import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.Path; import org.apache.flink.formats.avro.AvroInputFormat; +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.avro.file.DataFileWriter; import org.apache.avro.io.DatumWriter; @@ -140,6 +141,14 @@ public Tuple2 reduce( } } + private static final class KeyAssigner implements KeySelector, String> { + + @Override + public String getKey(Tuple2 record) throws Exception { + return record.f0; + } + } + // -------------------------------------------------------------------------------------------- // Test Data // -------------------------------------------------------------------------------------------- @@ -199,15 +208,15 @@ public static void writeTestData(File testFile, int numRecords) throws IOExcepti public static void main(String[] args) throws Exception { String inputPath = args[0]; - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet input = + DataStream input = env.createInput(new AvroInputFormat(new Path(inputPath), MyUser.class)); - DataSet> result = - input.map(new NameExtractor()).groupBy(0).reduce(new NameGrouper()); + DataStream> result = + input.map(new NameExtractor()).keyBy(new KeyAssigner()).reduce(new NameGrouper()); - result.output(new DiscardingOutputFormat>()); + result.sinkTo(new DiscardingSink<>()); env.execute(); } } diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeExtractionTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeExtractionTest.java index 1a02ae21224f7..de4710c2f692e 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeExtractionTest.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeExtractionTest.java @@ -18,26 +18,24 @@ package org.apache.flink.formats.avro.typeutils; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; 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.tuple.Tuple2; +import org.apache.flink.connector.file.sink.FileSink; import org.apache.flink.core.fs.Path; import org.apache.flink.formats.avro.AvroInputFormat; import org.apache.flink.formats.avro.AvroRecordInputFormatTest; -import org.apache.flink.formats.avro.generated.Fixed16; import org.apache.flink.formats.avro.generated.User; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.junit5.InjectMiniCluster; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.CollectionTestEnvironment; import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.test.util.TestEnvironment; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -45,7 +43,6 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.File; @@ -54,9 +51,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - /** Tests for the {@link AvroInputFormat} reading Pojos. */ class AvroTypeExtractionTest { @@ -90,13 +84,15 @@ public void after() throws Exception { @ValueSource(booleans = {true, false}) void testSimpleAvroRead(boolean useMiniCluster, @InjectMiniCluster MiniCluster miniCluster) throws Exception { - final ExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); + final StreamExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); Path in = new Path(inFile.getAbsoluteFile().toURI()); AvroInputFormat users = new AvroInputFormat<>(in, User.class); - DataSet usersDS = env.createInput(users).map((value) -> value); + DataStream usersDS = env.createInput(users).map((value) -> value); - usersDS.writeAsText(resultPath); + usersDS.sinkTo( + FileSink.forRowFormat(new Path(resultPath), new SimpleStringEncoder()) + .build()); env.execute("Simple Avro read job"); @@ -132,12 +128,12 @@ void testSimpleAvroRead(boolean useMiniCluster, @InjectMiniCluster MiniCluster m @ValueSource(booleans = {true, false}) void testSerializeWithAvro(boolean useMiniCluster, @InjectMiniCluster MiniCluster miniCluster) throws Exception { - final ExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); + final StreamExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); env.getConfig().getSerializerConfig().setForceAvro(true); Path in = new Path(inFile.getAbsoluteFile().toURI()); AvroInputFormat users = new AvroInputFormat<>(in, User.class); - DataSet usersDS = + DataStream usersDS = env.createInput(users) .map( (MapFunction) @@ -148,7 +144,9 @@ void testSerializeWithAvro(boolean useMiniCluster, @InjectMiniCluster MiniCluste return value; }); - usersDS.writeAsText(resultPath); + usersDS.sinkTo( + FileSink.forRowFormat(new Path(resultPath), new SimpleStringEncoder()) + .build()); env.execute("Simple Avro read job"); @@ -183,25 +181,24 @@ void testSerializeWithAvro(boolean useMiniCluster, @InjectMiniCluster MiniCluste @ValueSource(booleans = {true, false}) void testKeySelection(boolean useMiniCluster, @InjectMiniCluster MiniCluster miniCluster) throws Exception { - final ExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); + final StreamExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); env.getConfig().enableObjectReuse(); Path in = new Path(inFile.getAbsoluteFile().toURI()); AvroInputFormat users = new AvroInputFormat<>(in, User.class); - DataSet usersDS = env.createInput(users); - - DataSet> res = - usersDS.groupBy("name") - .reduceGroup( - (GroupReduceFunction>) - (values, out) -> { - for (User u : values) { - out.collect( - new Tuple2<>(u.getName().toString(), 1)); - } - }) + DataStream usersDS = env.createInput(users); + + DataStream> res = + usersDS.keyBy(User::getName) + .map( + (MapFunction>) + value -> new Tuple2<>(value.getName().toString(), 1)) .returns(Types.TUPLE(Types.STRING, Types.INT)); - res.writeAsText(resultPath); + res.sinkTo( + FileSink.forRowFormat( + new Path(resultPath), + new SimpleStringEncoder>()) + .build()); env.execute("Avro Key selection"); expected = "(Alyssa,1)\n(Charlie,1)\n"; @@ -211,28 +208,25 @@ void testKeySelection(boolean useMiniCluster, @InjectMiniCluster MiniCluster min @ValueSource(booleans = {true, false}) void testWithAvroGenericSer(boolean useMiniCluster, @InjectMiniCluster MiniCluster miniCluster) throws Exception { - final ExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); + final StreamExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); env.getConfig().getSerializerConfig().setForceAvro(true); Path in = new Path(inFile.getAbsoluteFile().toURI()); AvroInputFormat users = new AvroInputFormat<>(in, User.class); - DataSet usersDS = env.createInput(users); - - DataSet> res = - usersDS.groupBy( - (KeySelector) - value -> String.valueOf(value.getName())) - .reduceGroup( - (GroupReduceFunction>) - (values, out) -> { - for (User u : values) { - out.collect( - new Tuple2<>(u.getName().toString(), 1)); - } - }) + DataStreamSource usersDS = env.createInput(users); + + DataStream> res = + usersDS.keyBy(User::getName) + .map( + (MapFunction>) + value -> new Tuple2<>(value.getName().toString(), 1)) .returns(Types.TUPLE(Types.STRING, Types.INT)); - res.writeAsText(resultPath); + res.sinkTo( + FileSink.forRowFormat( + new Path(resultPath), + new SimpleStringEncoder>()) + .build()); env.execute("Avro Key selection"); expected = "(Charlie,1)\n(Alyssa,1)\n"; @@ -242,28 +236,25 @@ void testWithAvroGenericSer(boolean useMiniCluster, @InjectMiniCluster MiniClust @ValueSource(booleans = {true, false}) void testWithKryoGenericSer(boolean useMiniCluster, @InjectMiniCluster MiniCluster miniCluster) throws Exception { - final ExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); + final StreamExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); env.getConfig().getSerializerConfig().setForceKryo(true); Path in = new Path(inFile.getAbsoluteFile().toURI()); AvroInputFormat users = new AvroInputFormat<>(in, User.class); - DataSet usersDS = env.createInput(users); - - DataSet> res = - usersDS.groupBy( - (KeySelector) - value -> String.valueOf(value.getName())) - .reduceGroup( - (GroupReduceFunction>) - (values, out) -> { - for (User u : values) { - out.collect( - new Tuple2<>(u.getName().toString(), 1)); - } - }) + DataStreamSource usersDS = env.createInput(users); + + DataStream> res = + usersDS.keyBy(User::getName) + .map( + (MapFunction>) + value -> new Tuple2<>(value.getName().toString(), 1)) .returns(Types.TUPLE(Types.STRING, Types.INT)); - res.writeAsText(resultPath); + res.sinkTo( + FileSink.forRowFormat( + new Path(resultPath), + new SimpleStringEncoder>()) + .build()); env.execute("Avro Key selection"); expected = "(Charlie,1)\n(Alyssa,1)\n"; @@ -279,57 +270,10 @@ private static Stream testField() { Arguments.of(env, "type_double_test"))); } - /** Test some known fields for grouping on. */ - @ParameterizedTest - @MethodSource("testField") - void testField( - boolean useMiniCluster, - final String fieldName, - @InjectMiniCluster MiniCluster miniCluster) - throws Exception { - final ExecutionEnvironment env = getExecutionEnvironment(useMiniCluster, miniCluster); - Path in = new Path(inFile.getAbsoluteFile().toURI()); - - AvroInputFormat users = new AvroInputFormat<>(in, User.class); - DataSet usersDS = env.createInput(users); - - DataSet res = - usersDS.groupBy(fieldName) - .reduceGroup( - (GroupReduceFunction) - (values, out) -> { - for (User u : values) { - out.collect(u.get(fieldName)); - } - }) - .returns(Object.class); - res.writeAsText(resultPath); - env.execute("Simple Avro read job"); - - // test if automatic registration of the Types worked - ExecutionConfig ec = env.getConfig(); - assertThat(ec.getSerializerConfig().getRegisteredKryoTypes()).contains(Fixed16.class); - - switch (fieldName) { - case "name": - expected = "Alyssa\nCharlie"; - break; - case "type_enum": - expected = "GREEN\nRED\n"; - break; - case "type_double_test": - expected = "123.45\n1.337\n"; - break; - default: - fail("Unknown field"); - break; - } - } - - private static ExecutionEnvironment getExecutionEnvironment( + private static StreamExecutionEnvironment getExecutionEnvironment( boolean useMiniCluster, MiniCluster miniCluster) { return useMiniCluster - ? new TestEnvironment(miniCluster, PARALLELISM, false) - : new CollectionTestEnvironment(); + ? new TestStreamEnvironment(miniCluster, PARALLELISM) + : StreamExecutionEnvironment.getExecutionEnvironment(); } } diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml index 94dcf74953900..f7c2c6768d546 100644 --- a/flink-fs-tests/pom.xml +++ b/flink-fs-tests/pom.xml @@ -84,7 +84,7 @@ under the License. org.apache.flink - flink-examples-batch + flink-examples-streaming ${project.version} test diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java index 6ecb9e406781f..ae4b911f75425 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.io.FilePathFilter; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.ConfigConstants; @@ -29,10 +28,12 @@ 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.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator; import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperatorFactory; import org.apache.flink.streaming.api.functions.source.FileProcessingMode; import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit; import org.apache.flink.streaming.api.functions.source.legacy.ContinuousFileMonitoringFunction; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; import org.apache.flink.test.util.AbstractTestBaseJUnit4; import org.apache.flink.util.ExceptionUtils; diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java index 19d8707d1f396..2394c3e182695 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.FileInputSplit; @@ -35,6 +34,7 @@ import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit; import org.apache.flink.streaming.api.functions.source.legacy.ContinuousFileMonitoringFunction; import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java index 081da0d0006f9..d32d63bc06e66 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FilePathFilter; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.ConfigConstants; @@ -38,6 +37,7 @@ import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit; import org.apache.flink.streaming.api.functions.source.legacy.ContinuousFileMonitoringFunction; import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java index e9bbe0c5d3dec..3d73a43f15fef 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java @@ -21,21 +21,22 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.io.FirstAttemptInitializationContext; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.ExecutionEnvironmentFactory; -import org.apache.flink.api.java.LocalEnvironment; -import org.apache.flink.api.java.io.TextOutputFormat; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; -import org.apache.flink.examples.java.wordcount.WordCount; import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.blob.BlobUtils; import org.apache.flink.runtime.blob.TestingBlobHelpers; import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory; +import org.apache.flink.streaming.api.legacy.io.TextOutputFormat; +import org.apache.flink.streaming.examples.wordcount.WordCount; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.OperatingSystem; import org.apache.commons.io.IOUtils; @@ -56,6 +57,8 @@ import java.io.File; import java.io.IOException; import java.io.StringWriter; +import java.util.ArrayList; +import java.util.List; import static org.junit.Assert.assertTrue; @@ -135,7 +138,8 @@ public void testHDFS() { WordCount.main( new String[] { "--input", file.toString(), - "--output", result.toString() + "--output", result.toString(), + "--execution-mode", "BATCH" }); } catch (Throwable t) { t.printStackTrace(); @@ -147,13 +151,18 @@ public void testHDFS() { assertTrue("No result file present", hdfs.exists(result)); // validate output: - org.apache.hadoop.fs.FSDataInputStream inStream = hdfs.open(result); StringWriter writer = new StringWriter(); - IOUtils.copy(inStream, writer); + List fileStatusList = new ArrayList<>(); + getAllFileInDirectory(result, fileStatusList); + for (FileStatus fileStatus : fileStatusList) { + org.apache.hadoop.fs.FSDataInputStream inStream = hdfs.open(fileStatus.getPath()); + IOUtils.copy(inStream, writer); + inStream.close(); + } + String resultString = writer.toString(); - Assert.assertEquals("hdfs 10\n" + "hello 10\n", resultString); - inStream.close(); + Assert.assertEquals("(hdfs,10)\n" + "(hello,10)\n", resultString); } catch (IOException e) { e.printStackTrace(); @@ -293,17 +302,18 @@ public void testBlobCacheCorruptedFile() throws Exception { } } - abstract static class DopOneTestEnvironment extends ExecutionEnvironment { + abstract static class DopOneTestEnvironment extends StreamExecutionEnvironment { public static void setAsContext() { - final LocalEnvironment le = new LocalEnvironment(); + final LocalStreamEnvironment le = new LocalStreamEnvironment(); le.setParallelism(1); initializeContextEnvironment( - new ExecutionEnvironmentFactory() { + new StreamExecutionEnvironmentFactory() { @Override - public ExecutionEnvironment createExecutionEnvironment() { + public StreamExecutionEnvironment createExecutionEnvironment( + org.apache.flink.configuration.Configuration configuration) { return le; } }); @@ -313,4 +323,20 @@ public static void unsetAsContext() { resetContextEnvironment(); } } + + public void getAllFileInDirectory( + org.apache.hadoop.fs.Path hdfsDir, List fileStatusList) { + try { + FileStatus[] fileStatuses = hdfs.listStatus(hdfsDir); + for (FileStatus fileStatus : fileStatuses) { + if (fileStatus.isDirectory()) { + getAllFileInDirectory(fileStatus.getPath(), fileStatusList); + } else { + fileStatusList.add(fileStatus); + } + } + } catch (Exception e) { + ExceptionUtils.rethrow(e); + } + } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index 4b8c5f205a125..7d56f33d49092 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -87,6 +87,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.util.AbstractID; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.io.IOException; import java.util.ArrayList; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index eb0d7ec4143b4..ffb08dffec90f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -66,6 +66,7 @@ import org.apache.flink.util.NumberSequenceIterator; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SplittableIterator; +import org.apache.flink.util.Utils; import org.apache.flink.util.WrappingRuntimeException; import org.slf4j.Logger; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java index a289c340b6dcb..5b061b1c32c00 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java @@ -120,7 +120,8 @@ private static Configuration validateAndGetEffectiveConfiguration( final int port, final String[] jarFiles, final URL[] globalClasspaths) { - RemoteEnvironmentConfigUtils.validate(host, port); + // TODO: temporary hack this function, as flink-java module will be removed finally. + // RemoteEnvironmentConfigUtils.validate(host, port); return getEffectiveConfiguration( getClientConfiguration(configuration), host, @@ -146,9 +147,10 @@ private static Configuration getEffectiveConfiguration( final Configuration effectiveConfiguration = new Configuration(baseConfiguration); - RemoteEnvironmentConfigUtils.setJobManagerAddressToConfig( - host, port, effectiveConfiguration); - RemoteEnvironmentConfigUtils.setJarURLsToConfig(jars, effectiveConfiguration); + // TODO: temporary hack this function, as flink-java module will be removed finally. + // RemoteEnvironmentConfigUtils.setJobManagerAddressToConfig( + // host, port, effectiveConfiguration); + // RemoteEnvironmentConfigUtils.setJarURLsToConfig(jars, effectiveConfiguration); ConfigUtils.encodeCollectionToConfig( effectiveConfiguration, PipelineOptions.CLASSPATHS, classpaths, URL::toString); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java index b8a76ce5d8ce3..af45c9b3facdb 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java @@ -22,7 +22,6 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.operators.DataSource; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple1; @@ -55,6 +54,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.Path; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.ArrayList; import java.util.Arrays; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/DiscardingOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/DiscardingOutputFormat.java deleted file mode 100644 index 03ba917e07223..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/DiscardingOutputFormat.java +++ /dev/null @@ -1,52 +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.api.java.io; - -import org.apache.flink.annotation.Public; -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.configuration.Configuration; - -/** - * An output format that simply discards all elements. - * - * @param The type of the elements accepted by the output format. - * @deprecated 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 -public class DiscardingOutputFormat implements OutputFormat { - - private static final long serialVersionUID = 1L; - - @Override - public void configure(Configuration parameters) {} - - @Override - public void open(InitializationContext context) {} - - @Override - public void writeRecord(T record) {} - - @Override - public void close() {} -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index 689beaa5f88bd..c7be30cb30f0a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -35,7 +35,6 @@ import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder; @@ -45,6 +44,7 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.util.Utils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index 617b2da578388..760f49a5b07da 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple1; @@ -61,6 +60,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.Arrays; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index b4911abd861b8..5ae9105c41d1c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichFlatJoinFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; import org.apache.flink.api.common.operators.Keys; @@ -39,7 +40,6 @@ import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; import org.apache.flink.api.java.functions.KeySelector; @@ -51,7 +51,6 @@ import org.apache.flink.api.java.operators.translation.TupleLeftUnwrappingJoiner; import org.apache.flink.api.java.operators.translation.TupleRightUnwrappingJoiner; import org.apache.flink.api.java.operators.translation.TupleUnwrappingJoiner; -import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple10; @@ -82,6 +81,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Collector; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.Arrays; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index 8def3b0742404..fae3575962960 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -29,10 +29,10 @@ import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.FirstReducer; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.Arrays; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index 1bcfc457df0bb..71bfb45fda702 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -28,7 +28,6 @@ import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.functions.FirstReducer; import org.apache.flink.api.java.functions.KeySelector; @@ -37,6 +36,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; /** * A {@link Grouping} that is unsorted. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java index f127f2da33b82..2b6b3b0aeac48 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java @@ -26,12 +26,12 @@ import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin; import org.apache.flink.api.java.operators.JoinOperator.EquiJoin; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.util.Utils; /** * Intermediate step of an Outer Join transformation. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java index f062b7c8dedae..35cf925cfb077 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java @@ -20,9 +20,11 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; +import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java index c7143e00b255e..7715924b2563b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanFilterOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.FilterOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java index bae7a8f919495..4d7f1d49f8023 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java @@ -20,9 +20,11 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; +import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java index c867bae10df9c..bf064feeeaf23 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java @@ -20,9 +20,11 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; +import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java index b90600c40c30a..e63e1288b3a0c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java @@ -20,9 +20,11 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; +import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java index 2e4a26c1d4382..ce09d171f7bd8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java @@ -21,9 +21,11 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; +import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java index 7877dcd2d084e..bb2f6c062edf1 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java index ece3355218106..25344b4ac7298 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java index bb05160c1993a..dc2e507be0947 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java index b28e0756a2a5f..539b6abfd1b27 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleLeftUnwrappingJoiner.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java index 20db3ed72ddc1..2a01a84ff3e9e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleRightUnwrappingJoiner.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java index 212d312908519..5662988f3968f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingJoiner.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java index e85df068409f6..f196297ea1d45 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java.operators.translation; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java deleted file mode 100644 index 8ff6fa92725c9..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ /dev/null @@ -1,64 +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.api.java.operators.translation; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.functions.util.FunctionUtils; - -/** - * Wrapper around {@link Function}. - * - * @param - */ -@Internal -public abstract class WrappingFunction extends AbstractRichFunction { - - private static final long serialVersionUID = 1L; - - protected T wrappedFunction; - - protected WrappingFunction(T wrappedFunction) { - this.wrappedFunction = wrappedFunction; - } - - @Override - public void open(OpenContext openContext) throws Exception { - FunctionUtils.openFunction(this.wrappedFunction, openContext); - } - - @Override - public void close() throws Exception { - FunctionUtils.closeFunction(this.wrappedFunction); - } - - @Override - public void setRuntimeContext(RuntimeContext t) { - super.setRuntimeContext(t); - - FunctionUtils.setFunctionRuntimeContext(this.wrappedFunction, t); - } - - public T getWrappedFunction() { - return this.wrappedFunction; - } -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/FlinkChillPackageRegistrar.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/FlinkChillPackageRegistrar.java deleted file mode 100644 index 903b4ff9a1a6d..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/FlinkChillPackageRegistrar.java +++ /dev/null @@ -1,95 +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.api.java.typeutils.runtime.kryo; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import com.twitter.chill.java.ArraysAsListSerializer; -import com.twitter.chill.java.BitSetSerializer; -import com.twitter.chill.java.InetSocketAddressSerializer; -import com.twitter.chill.java.LocaleSerializer; -import com.twitter.chill.java.RegexSerializer; -import com.twitter.chill.java.SimpleDateFormatSerializer; -import com.twitter.chill.java.SqlDateSerializer; -import com.twitter.chill.java.SqlTimeSerializer; -import com.twitter.chill.java.TimestampSerializer; -import com.twitter.chill.java.URISerializer; -import com.twitter.chill.java.UUIDSerializer; - -import java.net.InetSocketAddress; -import java.net.URI; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.text.SimpleDateFormat; -import java.util.Arrays; -import java.util.BitSet; -import java.util.Locale; -import java.util.PriorityQueue; -import java.util.UUID; -import java.util.regex.Pattern; - -/** - * Registers all chill serializers used for Java types. - * - *

All registrations use a hard-coded ID which were determined at commit - * 18f176ce86900fd4e932c73f3d138912355c6880. - */ -public class FlinkChillPackageRegistrar implements ChillSerializerRegistrar { - - private static final int FIRST_REGISTRATION_ID = 73; - - @Override - public int getNextRegistrationId() { - return 85; - } - - @Override - public void registerSerializers(Kryo kryo) { - //noinspection ArraysAsListWithZeroOrOneArgument - new RegistrationHelper(FIRST_REGISTRATION_ID, kryo) - .register(Arrays.asList("").getClass(), new ArraysAsListSerializer()) - .register(BitSet.class, new BitSetSerializer()) - .register(PriorityQueue.class, new PriorityQueueSerializer()) - .register(Pattern.class, new RegexSerializer()) - .register(Date.class, new SqlDateSerializer()) - .register(Time.class, new SqlTimeSerializer()) - .register(Timestamp.class, new TimestampSerializer()) - .register(URI.class, new URISerializer()) - .register(InetSocketAddress.class, new InetSocketAddressSerializer()) - .register(UUID.class, new UUIDSerializer()) - .register(Locale.class, new LocaleSerializer()) - .register(SimpleDateFormat.class, new SimpleDateFormatSerializer()); - } - - private static final class RegistrationHelper { - private int nextRegistrationId; - private final Kryo kryo; - - public RegistrationHelper(int firstRegistrationId, Kryo kryo) { - this.nextRegistrationId = firstRegistrationId; - this.kryo = kryo; - } - - public RegistrationHelper register(Class type, Serializer serializer) { - kryo.register(type, serializer, nextRegistrationId++); - return this; - } - } -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/PriorityQueueSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/PriorityQueueSerializer.java deleted file mode 100644 index e33d6d9dcca04..0000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/PriorityQueueSerializer.java +++ /dev/null @@ -1,90 +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.api.java.typeutils.runtime.kryo; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import com.twitter.chill.IKryoRegistrar; -import com.twitter.chill.SingleRegistrar; - -import java.lang.reflect.Field; -import java.util.Comparator; -import java.util.PriorityQueue; - -/* -This code is copied as is from Twitter Chill 0.7.4 because we need to user a newer chill version -but want to ensure that the serializers that are registered by default stay the same. - -The only changes to the code are those that are required to make it compile and pass checkstyle -checks in our code base. - */ - -class PriorityQueueSerializer extends Serializer> { - private Field compField; - - public static IKryoRegistrar registrar() { - return new SingleRegistrar(PriorityQueue.class, new PriorityQueueSerializer()); - } - - public PriorityQueueSerializer() { - try { - compField = PriorityQueue.class.getDeclaredField("comparator"); - compField.setAccessible(true); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public Comparator getComparator(PriorityQueue q) { - try { - return (Comparator) compField.get(q); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void write(Kryo k, Output o, PriorityQueue q) { - k.writeClassAndObject(o, getComparator(q)); - o.writeInt(q.size(), true); - for (Object a : q) { - k.writeClassAndObject(o, a); - o.flush(); - } - } - - public PriorityQueue read(Kryo k, Input i, Class> c) { - Comparator comp = (Comparator) k.readClassAndObject(i); - int sz = i.readInt(true); - // can't create with size 0: - PriorityQueue result; - if (sz == 0) { - result = new PriorityQueue(1, comp); - } else { - result = new PriorityQueue(sz, comp); - } - int idx = 0; - while (idx < sz) { - result.add(k.readClassAndObject(i)); - idx += 1; - } - return result; - } -} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java index edb278031e806..92e6bcbd96554 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java @@ -30,7 +30,6 @@ import org.apache.flink.api.common.operators.base.PartitionOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SampleInCoordinator; import org.apache.flink.api.java.functions.SampleInPartition; @@ -46,6 +45,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.AbstractID; import org.apache.flink.util.Collector; +import org.apache.flink.util.Utils; import java.util.ArrayList; import java.util.Collections; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java index 64d328675f9db..ef06291bbbff1 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java @@ -102,33 +102,33 @@ private Plan createPlan() { * @param plan the generated plan. */ private void registerGenericTypeInfoIfConfigured(Plan plan) { - if (!config.isAutoTypeRegistrationDisabled()) { - plan.accept( - new Visitor>() { - - private final Set> registeredTypes = new HashSet<>(); - private final Set> - visitedOperators = new HashSet<>(); - - @Override - public boolean preVisit( - org.apache.flink.api.common.operators.Operator visitable) { - if (!visitedOperators.add(visitable)) { - return false; - } - OperatorInformation opInfo = visitable.getOperatorInfo(); - Serializers.recursivelyRegisterType( - opInfo.getOutputType(), - config.getSerializerConfig(), - registeredTypes); - return true; + // TODO: The flink-java module will totally removed, temporary hack this function for code + // clean + plan.accept( + new Visitor>() { + + private final Set> registeredTypes = new HashSet<>(); + private final Set> + visitedOperators = new HashSet<>(); + + @Override + public boolean preVisit( + org.apache.flink.api.common.operators.Operator visitable) { + if (!visitedOperators.add(visitable)) { + return false; } - - @Override - public void postVisit( - org.apache.flink.api.common.operators.Operator visitable) {} - }); - } + OperatorInformation opInfo = visitable.getOperatorInfo(); + Serializers.recursivelyRegisterType( + opInfo.getOutputType(), + config.getSerializerConfig(), + registeredTypes); + return true; + } + + @Override + public void postVisit( + org.apache.flink.api.common.operators.Operator visitable) {} + }); } private void registerCachedFiles(Plan plan) { diff --git a/flink-java/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java b/flink-java/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java deleted file mode 100644 index effa0fb3b179b..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/io/SequentialFormatTestBase.java +++ /dev/null @@ -1,293 +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.api.common.io; - -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.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; -import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; -import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -import java.io.File; -import java.io.IOException; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Test base for {@link BinaryInputFormat} and {@link BinaryOutputFormat}. */ -@ExtendWith(ParameterizedTestExtension.class) -public abstract class SequentialFormatTestBase { - - private static class InputSplitSorter implements Comparator { - @Override - public int compare(FileInputSplit o1, FileInputSplit o2) { - int pathOrder = o1.getPath().getName().compareTo(o2.getPath().getName()); - return pathOrder == 0 ? Long.signum(o1.getStart() - o2.getStart()) : pathOrder; - } - } - - @Parameter public int numberOfTuples; - - @Parameter(value = 1) - public long blockSize; - - @Parameter(value = 2) - public int parallelism; - - @Parameter(value = 3) - public int[] rawDataSizes; - - protected File tempFile; - - /** Count how many bytes would be written if all records were directly serialized. */ - @BeforeEach - void calcRawDataSize() throws IOException { - int recordIndex = 0; - for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { - ByteCounter byteCounter = new ByteCounter(); - - for (int fileCount = 0; - fileCount < this.getNumberOfTuplesPerFile(fileIndex); - fileCount++, recordIndex++) { - writeRecord( - this.getRecord(recordIndex), new DataOutputViewStreamWrapper(byteCounter)); - } - this.rawDataSizes[fileIndex] = byteCounter.getLength(); - } - } - - /** Checks if the expected input splits were created. */ - @TestTemplate - void checkInputSplits() throws IOException { - FileInputSplit[] inputSplits = this.createInputFormat().createInputSplits(0); - Arrays.sort(inputSplits, new InputSplitSorter()); - - int splitIndex = 0; - for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { - List sameFileSplits = new ArrayList(); - Path lastPath = inputSplits[splitIndex].getPath(); - for (; splitIndex < inputSplits.length; splitIndex++) { - if (!inputSplits[splitIndex].getPath().equals(lastPath)) { - break; - } - sameFileSplits.add(inputSplits[splitIndex]); - } - - assertThat(this.getExpectedBlockCount(fileIndex)).isEqualTo(sameFileSplits.size()); - - long lastBlockLength = - this.rawDataSizes[fileIndex] % (this.blockSize - getInfoSize()) + getInfoSize(); - for (int index = 0; index < sameFileSplits.size(); index++) { - assertThat(this.blockSize * index).isEqualTo(sameFileSplits.get(index).getStart()); - if (index < sameFileSplits.size() - 1) { - assertThat(this.blockSize).isEqualTo(sameFileSplits.get(index).getLength()); - } - } - assertThat(lastBlockLength) - .isEqualTo(sameFileSplits.get(sameFileSplits.size() - 1).getLength()); - } - } - - /** Tests if the expected sequence and amount of data can be read. */ - @TestTemplate - void checkRead() throws Exception { - BinaryInputFormat input = this.createInputFormat(); - FileInputSplit[] inputSplits = input.createInputSplits(0); - Arrays.sort(inputSplits, new InputSplitSorter()); - - int readCount = 0; - - for (FileInputSplit inputSplit : inputSplits) { - input.open(inputSplit); - input.reopen(inputSplit, input.getCurrentState()); - - T record = createInstance(); - - while (!input.reachedEnd()) { - if (input.nextRecord(record) != null) { - this.checkEquals(this.getRecord(readCount), record); - - if (!input.reachedEnd()) { - Tuple2 state = input.getCurrentState(); - - input = this.createInputFormat(); - input.reopen(inputSplit, state); - } - readCount++; - } - } - } - assertThat(this.numberOfTuples).isEqualTo(readCount); - } - - /** Tests the statistics of the given format. */ - @TestTemplate - void checkStatistics() { - BinaryInputFormat input = this.createInputFormat(); - BaseStatistics statistics = input.getStatistics(null); - assertThat(this.numberOfTuples).isEqualTo(statistics.getNumberOfRecords()); - } - - @AfterEach - void cleanup() { - this.deleteRecursively(this.tempFile); - } - - private void deleteRecursively(File file) { - if (file.isDirectory()) { - for (File subFile : file.listFiles()) { - this.deleteRecursively(subFile); - } - } else { - file.delete(); - } - } - - /** Write out the tuples in a temporary file and return it. */ - @BeforeEach - void writeTuples() throws IOException { - this.tempFile = File.createTempFile("BinaryInputFormat", null); - this.tempFile.deleteOnExit(); - Configuration configuration = new Configuration(); - if (this.parallelism == 1) { - BinaryOutputFormat output = - createOutputFormat(this.tempFile.toURI().toString(), configuration); - for (int index = 0; index < this.numberOfTuples; index++) { - output.writeRecord(this.getRecord(index)); - } - output.close(); - } else { - this.tempFile.delete(); - this.tempFile.mkdir(); - int recordIndex = 0; - for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { - BinaryOutputFormat output = - createOutputFormat( - this.tempFile.toURI() + "/" + (fileIndex + 1), configuration); - for (int fileCount = 0; - fileCount < this.getNumberOfTuplesPerFile(fileIndex); - fileCount++, recordIndex++) { - output.writeRecord(this.getRecord(recordIndex)); - } - output.close(); - } - } - } - - private int getNumberOfTuplesPerFile(int fileIndex) { - return this.numberOfTuples / this.parallelism; - } - - /** Tests if the length of the file matches the expected value. */ - @TestTemplate - void checkLength() { - File[] files = - this.tempFile.isDirectory() - ? this.tempFile.listFiles() - : new File[] {this.tempFile}; - Arrays.sort(files); - for (int fileIndex = 0; fileIndex < this.parallelism; fileIndex++) { - long lastBlockLength = this.rawDataSizes[fileIndex] % (this.blockSize - getInfoSize()); - long expectedLength = - (this.getExpectedBlockCount(fileIndex) - 1) * this.blockSize - + getInfoSize() - + lastBlockLength; - assertThat(expectedLength).isEqualTo(files[fileIndex].length()); - } - } - - protected abstract BinaryInputFormat createInputFormat(); - - protected abstract BinaryOutputFormat createOutputFormat( - String path, Configuration configuration) throws IOException; - - protected abstract int getInfoSize(); - - /** Returns the record to write at the given position. */ - protected abstract T getRecord(int index); - - protected abstract T createInstance(); - - protected abstract void writeRecord(T record, DataOutputView outputView) throws IOException; - - /** Checks if both records are equal. */ - protected abstract void checkEquals(T expected, T actual); - - private int getExpectedBlockCount(int fileIndex) { - int expectedBlockCount = - (int) - Math.ceil( - (double) this.rawDataSizes[fileIndex] - / (this.blockSize - getInfoSize())); - return expectedBlockCount; - } - - @Parameters - public static List getParameters() { - ArrayList params = new ArrayList<>(); - for (int parallelism = 1; parallelism <= 2; parallelism++) { - // numberOfTuples, blockSize, parallelism - params.add( - new Object[] { - 100, BinaryOutputFormat.NATIVE_BLOCK_SIZE, parallelism, new int[parallelism] - }); - params.add( - new Object[] { - 10000, - BinaryOutputFormat.NATIVE_BLOCK_SIZE, - parallelism, - new int[parallelism] - }); - } - return params; - } - - /** Counts the bytes that would be written. */ - private static final class ByteCounter extends OutputStream { - int length = 0; - - /** - * Returns the length. - * - * @return the length - */ - public int getLength() { - return this.length; - } - - @Override - public void write(int b) throws IOException { - this.length++; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java deleted file mode 100644 index d969d07d1d3b5..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/io/SerializedFormatTest.java +++ /dev/null @@ -1,97 +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.api.common.io; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.Record; -import org.apache.flink.types.StringValue; - -import org.junit.jupiter.api.BeforeEach; - -import java.io.IOException; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for serialized formats. */ -class SerializedFormatTest extends SequentialFormatTestBase { - - private BlockInfo info; - - @BeforeEach - void setup() { - info = createInputFormat().createBlockInfo(); - } - - @Override - protected BinaryInputFormat createInputFormat() { - Configuration configuration = new Configuration(); - - final SerializedInputFormat inputFormat = new SerializedInputFormat(); - inputFormat.setFilePath(this.tempFile.toURI().toString()); - - inputFormat.configure(configuration); - return inputFormat; - } - - @Override - protected BinaryOutputFormat createOutputFormat( - String path, Configuration configuration) throws IOException { - final SerializedOutputFormat outputFormat = new SerializedOutputFormat(); - outputFormat.setOutputFilePath(new Path(path)); - outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE); - - configuration = configuration == null ? new Configuration() : configuration; - outputFormat.configure(configuration); - outputFormat.open(FirstAttemptInitializationContext.of(0, 1)); - return outputFormat; - } - - @Override - protected int getInfoSize() { - return info.getInfoSize(); - } - - @Override - protected Record getRecord(int index) { - return new Record(new IntValue(index), new StringValue(String.valueOf(index))); - } - - @Override - protected Record createInstance() { - return new Record(); - } - - @Override - protected void writeRecord(Record record, DataOutputView outputView) throws IOException { - record.write(outputView); - } - - @Override - protected void checkEquals(Record expected, Record actual) { - assertThat(actual.getNumFields()).isEqualTo(expected.getNumFields()); - assertThat(actual.getField(0, IntValue.class)) - .isEqualTo(expected.getField(0, IntValue.class)); - assertThat((CharSequence) actual.getField(1, StringValue.class)) - .isEqualTo(expected.getField(1, StringValue.class)); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java deleted file mode 100644 index 7e8f3f0a87b14..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java +++ /dev/null @@ -1,76 +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.api.common.operators; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.accumulators.IntCounter; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CollectionExecutor} with accumulators. */ -class CollectionExecutionAccumulatorsTest { - - private static final String ACCUMULATOR_NAME = "TEST ACC"; - - @Test - void testAccumulator() { - try { - final int numElements = 100; - - ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - env.generateSequence(1, numElements) - .map(new CountingMapper()) - .output(new DiscardingOutputFormat()); - - JobExecutionResult result = env.execute(); - - assertThat(result.getNetRuntime()).isGreaterThanOrEqualTo(0); - - assertThat((int) result.getAccumulatorResult(ACCUMULATOR_NAME)).isEqualTo(numElements); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @SuppressWarnings("serial") - private static class CountingMapper extends RichMapFunction { - - private IntCounter accumulator; - - @Override - public void open(OpenContext openContext) { - accumulator = getRuntimeContext().getIntCounter(ACCUMULATOR_NAME); - } - - @Override - public Long map(Long value) { - accumulator.add(1); - return value; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionIterationTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionIterationTest.java deleted file mode 100644 index ddb285369c4c0..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionIterationTest.java +++ /dev/null @@ -1,181 +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.api.common.operators; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CollectionExecutor} with iterations. */ -@SuppressWarnings("serial") -class CollectionExecutionIterationTest implements Serializable { - - @Test - void testBulkIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - IterativeDataSet iteration = env.fromElements(1).iterate(10); - - DataSet result = - iteration.closeWith(iteration.map(new AddSuperstepNumberMapper())); - - List collected = new ArrayList<>(); - result.output(new LocalCollectionOutputFormat<>(collected)); - - env.execute(); - - assertThat(collected).containsExactly(56); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testBulkIterationWithTerminationCriterion() { - try { - ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - IterativeDataSet iteration = env.fromElements(1).iterate(100); - - DataSet iterationResult = iteration.map(new AddSuperstepNumberMapper()); - - DataSet terminationCriterion = - iterationResult.filter((FilterFunction) value -> value < 50); - - List collected = new ArrayList<>(); - - iteration - .closeWith(iterationResult, terminationCriterion) - .output(new LocalCollectionOutputFormat<>(collected)); - - env.execute(); - - assertThat(collected).containsExactly(56); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testDeltaIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> solInput = - env.fromElements( - new Tuple2<>(1, 0), - new Tuple2<>(2, 0), - new Tuple2<>(3, 0), - new Tuple2<>(4, 0)); - - @SuppressWarnings("unchecked") - DataSet> workInput = - env.fromElements( - new Tuple1<>(1), new Tuple1<>(2), new Tuple1<>(3), new Tuple1<>(4)); - - // Perform a delta iteration where we add those values to the workset where - // the second tuple field is smaller than the first tuple field. - // At the end both tuple fields must be the same. - - DeltaIteration, Tuple1> iteration = - solInput.iterateDelta(workInput, 10, 0); - - DataSet> solDelta = - iteration - .getSolutionSet() - .join(iteration.getWorkset()) - .where(0) - .equalTo(0) - .with( - new JoinFunction< - Tuple2, - Tuple1, - Tuple2>() { - @Override - public Tuple2 join( - Tuple2 first, - Tuple1 second) - throws Exception { - return new Tuple2( - first.f0, first.f1 + 1); - } - }); - - DataSet> nextWorkset = - solDelta.flatMap( - new FlatMapFunction, Tuple1>() { - @Override - public void flatMap( - Tuple2 in, Collector> out) - throws Exception { - if (in.f1 < in.f0) { - out.collect(new Tuple1<>(in.f0)); - } - } - }); - - List> collected = new ArrayList<>(); - - iteration - .closeWith(solDelta, nextWorkset) - .output(new LocalCollectionOutputFormat<>(collected)); - - env.execute(); - - // verify that both tuple fields are now the same - for (Tuple2 t : collected) { - assertThat(t.f1).isEqualTo(t.f0); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private static class AddSuperstepNumberMapper extends RichMapFunction { - - @Override - public Integer map(Integer value) { - int superstep = getIterationRuntimeContext().getSuperstepNumber(); - return value + superstep; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java deleted file mode 100644 index 354f78bb9fa2f..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java +++ /dev/null @@ -1,127 +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.api.common.operators; - -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichCrossFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CollectionExecutor} with broadcast variables. */ -@SuppressWarnings("serial") -class CollectionExecutionWithBroadcastVariableTest { - - private static final String BC_VAR_NAME = "BC"; - - private static final String[] TEST_DATA = {"A", "B", "C", "D"}; - private static final String SUFFIX = "-suffixed"; - - @Test - void testUnaryOp() { - try { - ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - DataSet bcData = env.fromElements(SUFFIX); - - List result = new ArrayList<>(); - - env.fromElements(TEST_DATA) - .map(new SuffixAppender()) - .withBroadcastSet(bcData, BC_VAR_NAME) - .output(new LocalCollectionOutputFormat<>(result)); - - env.execute(); - - assertThat(result).hasSameSizeAs(TEST_DATA); - for (String s : result) { - assertThat(s.indexOf(SUFFIX)).isGreaterThan(0); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testBinaryOp() { - try { - ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); - - DataSet bcData = env.fromElements(SUFFIX); - DataSet inData = env.fromElements(TEST_DATA); - - List result = new ArrayList<>(); - - inData.cross(inData) - .with(new SuffixCross()) - .withBroadcastSet(bcData, BC_VAR_NAME) - .output(new LocalCollectionOutputFormat<>(result)); - - env.execute(); - - assertThat(result).hasSize(TEST_DATA.length * TEST_DATA.length); - for (String s : result) { - assertThat(s.indexOf(SUFFIX)).isEqualTo(2); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private static final class SuffixAppender extends RichMapFunction { - - private String suffix; - - @Override - public void open(OpenContext openContext) { - suffix = getRuntimeContext().getBroadcastVariable(BC_VAR_NAME).get(0); - } - - @Override - public String map(String value) { - return value + suffix; - } - } - - private static final class SuffixCross extends RichCrossFunction { - - private String suffix; - - @Override - public void open(OpenContext openContext) { - suffix = getRuntimeContext().getBroadcastVariable(BC_VAR_NAME).get(0); - } - - @Override - public String cross(String s1, String s2) { - return s1 + s2 + suffix; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java deleted file mode 100644 index 840d821c71b15..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java +++ /dev/null @@ -1,228 +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.api.common.operators.base; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; -import org.apache.flink.api.common.TaskInfoImpl; -import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichCoGroupFunction; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.functions.util.RuntimeUDFContext; -import org.apache.flink.api.common.operators.BinaryOperatorInformation; -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.tuple.builder.Tuple2Builder; -import org.apache.flink.core.fs.Path; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Future; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CoGroupOperatorBase} on collections. */ -@SuppressWarnings("serial") -class CoGroupOperatorCollectionTest implements Serializable { - - @Test - void testExecuteOnCollection() { - try { - List> input1 = - Arrays.asList( - new Tuple2Builder() - .add("foo", 1) - .add("foobar", 1) - .add("foo", 1) - .add("bar", 1) - .add("foo", 1) - .add("foo", 1) - .build()); - - List> input2 = - Arrays.asList( - new Tuple2Builder() - .add("foo", 1) - .add("foo", 1) - .add("bar", 1) - .add("foo", 1) - .add("barfoo", 1) - .add("foo", 1) - .build()); - - ExecutionConfig executionConfig = new ExecutionConfig(); - final HashMap> accumulators = new HashMap<>(); - final HashMap> cpTasks = new HashMap<>(); - final TaskInfo taskInfo = new TaskInfoImpl("Test UDF", 4, 0, 4, 0); - final RuntimeContext ctx = - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - cpTasks, - accumulators, - UnregisteredMetricsGroup.createOperatorMetricGroup()); - - { - SumCoGroup udf1 = new SumCoGroup(); - SumCoGroup udf2 = new SumCoGroup(); - - executionConfig.disableObjectReuse(); - List> resultSafe = - getCoGroupOperator(udf1) - .executeOnCollections(input1, input2, ctx, executionConfig); - executionConfig.enableObjectReuse(); - List> resultRegular = - getCoGroupOperator(udf2) - .executeOnCollections(input1, input2, ctx, executionConfig); - - assertThat(udf1.isClosed).isTrue(); - assertThat(udf2.isClosed).isTrue(); - - Set> expected = - new HashSet<>( - Arrays.asList( - new Tuple2Builder() - .add("foo", 8) - .add("bar", 2) - .add("foobar", 1) - .add("barfoo", 1) - .build())); - - assertThat(new HashSet<>(resultSafe)).containsExactlyInAnyOrderElementsOf(expected); - assertThat(new HashSet<>(resultRegular)) - .containsExactlyInAnyOrderElementsOf(expected); - } - - { - executionConfig.disableObjectReuse(); - List> resultSafe = - getCoGroupOperator(new SumCoGroup()) - .executeOnCollections( - Collections.emptyList(), - Collections.emptyList(), - ctx, - executionConfig); - - executionConfig.enableObjectReuse(); - List> resultRegular = - getCoGroupOperator(new SumCoGroup()) - .executeOnCollections( - Collections.emptyList(), - Collections.emptyList(), - ctx, - executionConfig); - - assertThat(resultSafe).isEmpty(); - assertThat(resultRegular).isEmpty(); - } - } catch (Throwable t) { - t.printStackTrace(); - fail(t.getMessage()); - } - } - - private static class SumCoGroup - extends RichCoGroupFunction< - Tuple2, Tuple2, Tuple2> { - - private boolean isOpened = false; - private boolean isClosed = false; - - @Override - public void open(OpenContext openContext) throws Exception { - isOpened = true; - - RuntimeContext ctx = getRuntimeContext(); - assertThat(ctx.getTaskInfo().getTaskName()).isEqualTo("Test UDF"); - assertThat(ctx.getTaskInfo().getNumberOfParallelSubtasks()).isEqualTo(4); - assertThat(ctx.getTaskInfo().getIndexOfThisSubtask()).isZero(); - } - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception { - - assertThat(isOpened).isTrue(); - assertThat(isClosed).isFalse(); - - String f0 = null; - int sumF1 = 0; - - for (Tuple2 input : first) { - f0 = (f0 == null) ? input.f0 : f0; - sumF1 += input.f1; - } - - for (Tuple2 input : second) { - f0 = (f0 == null) ? input.f0 : f0; - sumF1 += input.f1; - } - - out.collect(Tuple2.of(f0, sumF1)); - } - - @Override - public void close() throws Exception { - isClosed = true; - } - } - - private CoGroupOperatorBase< - Tuple2, - Tuple2, - Tuple2, - CoGroupFunction< - Tuple2, - Tuple2, - Tuple2>> - getCoGroupOperator( - RichCoGroupFunction< - Tuple2, - Tuple2, - Tuple2> - udf) { - - TypeInformation> tuple2Info = - TypeInformation.of(new TypeHint>() {}); - - return new CoGroupOperatorBase<>( - udf, - new BinaryOperatorInformation<>(tuple2Info, tuple2Info, tuple2Info), - new int[] {0}, - new int[] {0}, - "coGroup on Collections"); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java deleted file mode 100644 index 6da2128404170..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java +++ /dev/null @@ -1,227 +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.api.common.operators.base; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; -import org.apache.flink.api.common.TaskInfoImpl; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.functions.util.RuntimeUDFContext; -import org.apache.flink.api.common.operators.UnaryOperatorInformation; -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.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; - -import static java.util.Arrays.asList; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link GroupReduceFunction}. */ -@SuppressWarnings({"serial", "unchecked"}) -class GroupReduceOperatorTest implements Serializable { - - private static final TypeInformation> STRING_INT_TUPLE = - TypeInformation.of(new TypeHint>() {}); - - @Test - void testGroupReduceCollection() { - try { - final GroupReduceFunction, Tuple2> reducer = - (values, out) -> { - Iterator> input = values.iterator(); - - Tuple2 result = input.next(); - int sum = result.f1; - while (input.hasNext()) { - Tuple2 next = input.next(); - sum += next.f1; - } - result.f1 = sum; - out.collect(result); - }; - - GroupReduceOperatorBase< - Tuple2, - Tuple2, - GroupReduceFunction, Tuple2>> - op = - new GroupReduceOperatorBase<>( - reducer, - new UnaryOperatorInformation<>( - STRING_INT_TUPLE, STRING_INT_TUPLE), - new int[] {0}, - "TestReducer"); - - List> input = - new ArrayList<>( - asList( - new Tuple2<>("foo", 1), - new Tuple2<>("foo", 3), - new Tuple2<>("bar", 2), - new Tuple2<>("bar", 4))); - - ExecutionConfig executionConfig = new ExecutionConfig(); - executionConfig.disableObjectReuse(); - List> resultMutableSafe = - op.executeOnCollections(input, null, executionConfig); - executionConfig.enableObjectReuse(); - List> resultRegular = - op.executeOnCollections(input, null, executionConfig); - - Set> resultSetMutableSafe = new HashSet<>(resultMutableSafe); - Set> resultSetRegular = new HashSet<>(resultRegular); - - Set> expectedResult = - new HashSet<>(asList(new Tuple2<>("foo", 4), new Tuple2<>("bar", 6))); - - assertThat(resultSetMutableSafe).isEqualTo(expectedResult); - assertThat(resultSetRegular).isEqualTo(expectedResult); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testGroupReduceCollectionWithRuntimeContext() { - try { - final String taskName = "Test Task"; - final AtomicBoolean opened = new AtomicBoolean(); - final AtomicBoolean closed = new AtomicBoolean(); - - final RichGroupReduceFunction, Tuple2> - reducer = - new RichGroupReduceFunction< - Tuple2, Tuple2>() { - - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - Iterator> input = values.iterator(); - - Tuple2 result = input.next(); - int sum = result.f1; - while (input.hasNext()) { - Tuple2 next = input.next(); - sum += next.f1; - } - result.f1 = sum; - out.collect(result); - } - - @Override - public void open(OpenContext openContext) throws Exception { - opened.set(true); - RuntimeContext ctx = getRuntimeContext(); - assertThat(ctx.getTaskInfo().getIndexOfThisSubtask()).isZero(); - assertThat(ctx.getTaskInfo().getNumberOfParallelSubtasks()) - .isOne(); - assertThat(ctx.getTaskInfo().getTaskName()).isEqualTo(taskName); - } - - @Override - public void close() throws Exception { - closed.set(true); - } - }; - - GroupReduceOperatorBase< - Tuple2, - Tuple2, - GroupReduceFunction, Tuple2>> - op = - new GroupReduceOperatorBase<>( - reducer, - new UnaryOperatorInformation<>( - STRING_INT_TUPLE, STRING_INT_TUPLE), - new int[] {0}, - "TestReducer"); - - List> input = - new ArrayList<>( - asList( - new Tuple2<>("foo", 1), - new Tuple2<>("foo", 3), - new Tuple2<>("bar", 2), - new Tuple2<>("bar", 4))); - - final TaskInfo taskInfo = new TaskInfoImpl(taskName, 1, 0, 1, 0); - - ExecutionConfig executionConfig = new ExecutionConfig(); - executionConfig.disableObjectReuse(); - List> resultMutableSafe = - op.executeOnCollections( - input, - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - new HashMap<>(), - new HashMap<>(), - UnregisteredMetricsGroup.createOperatorMetricGroup()), - executionConfig); - - executionConfig.enableObjectReuse(); - List> resultRegular = - op.executeOnCollections( - input, - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - new HashMap<>(), - new HashMap<>(), - UnregisteredMetricsGroup.createOperatorMetricGroup()), - executionConfig); - - Set> resultSetMutableSafe = new HashSet<>(resultMutableSafe); - Set> resultSetRegular = new HashSet<>(resultRegular); - - Set> expectedResult = - new HashSet<>(asList(new Tuple2<>("foo", 4), new Tuple2<>("bar", 6))); - - assertThat(resultSetMutableSafe).isEqualTo(expectedResult); - assertThat(resultSetRegular).isEqualTo(expectedResult); - - assertThat(opened.get()).isTrue(); - assertThat(closed.get()).isTrue(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java deleted file mode 100644 index 69be02873f02a..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java +++ /dev/null @@ -1,155 +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.api.common.operators.base; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; -import org.apache.flink.api.common.TaskInfoImpl; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.util.RuntimeUDFContext; -import org.apache.flink.api.common.operators.BinaryOperatorInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link InnerJoinOperatorBase}. */ -@SuppressWarnings({"unchecked", "serial"}) -class InnerJoinOperatorBaseTest implements Serializable { - - @Test - void testTupleBaseJoiner() { - final FlatJoinFunction< - Tuple3, - Tuple2, - Tuple2> - joiner = - (first, second, out) -> { - assertThat(second.f1).isEqualTo(first.f0); - assertThat(second.f0).isEqualTo(first.f2); - - out.collect(new Tuple2<>(first.f1, second.f0.toString())); - }; - - final TupleTypeInfo> leftTypeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, Double.class, Integer.class); - final TupleTypeInfo> rightTypeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(Integer.class, String.class); - final TupleTypeInfo> outTypeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(Double.class, String.class); - - final int[] leftKeys = new int[] {0, 2}; - final int[] rightKeys = new int[] {1, 0}; - - final String taskName = "Collection based tuple joiner"; - - final BinaryOperatorInformation< - Tuple3, - Tuple2, - Tuple2> - binaryOpInfo = - new BinaryOperatorInformation<>(leftTypeInfo, rightTypeInfo, outTypeInfo); - - final InnerJoinOperatorBase< - Tuple3, - Tuple2, - Tuple2, - FlatJoinFunction< - Tuple3, - Tuple2, - Tuple2>> - base = - new InnerJoinOperatorBase<>( - joiner, binaryOpInfo, leftKeys, rightKeys, taskName); - - final List> inputData1 = - new ArrayList<>( - Arrays.asList( - new Tuple3<>("foo", 42.0, 1), - new Tuple3<>("bar", 1.0, 2), - new Tuple3<>("bar", 2.0, 3), - new Tuple3<>("foobar", 3.0, 4), - new Tuple3<>("bar", 3.0, 3))); - - final List> inputData2 = - new ArrayList<>( - Arrays.asList( - new Tuple2<>(3, "bar"), - new Tuple2<>(4, "foobar"), - new Tuple2<>(2, "foo"))); - final Set> expected = - new HashSet<>( - Arrays.asList( - new Tuple2<>(2.0, "3"), - new Tuple2<>(3.0, "3"), - new Tuple2<>(3.0, "4"))); - - try { - final TaskInfo taskInfo = new TaskInfoImpl("op", 1, 0, 1, 0); - ExecutionConfig executionConfig = new ExecutionConfig(); - - executionConfig.disableObjectReuse(); - List> resultSafe = - base.executeOnCollections( - inputData1, - inputData2, - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - new HashMap<>(), - new HashMap<>(), - UnregisteredMetricsGroup.createOperatorMetricGroup()), - executionConfig); - - executionConfig.enableObjectReuse(); - List> resultRegular = - base.executeOnCollections( - inputData1, - inputData2, - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - new HashMap<>(), - new HashMap<>(), - UnregisteredMetricsGroup.createOperatorMetricGroup()), - executionConfig); - - assertThat(resultSafe).containsExactlyInAnyOrderElementsOf(expected); - assertThat(resultRegular).containsExactlyInAnyOrderElementsOf(expected); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java deleted file mode 100644 index b941db579876d..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java +++ /dev/null @@ -1,198 +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.api.common.operators.base; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; -import org.apache.flink.api.common.TaskInfoImpl; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.functions.RichReduceFunction; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.functions.util.RuntimeUDFContext; -import org.apache.flink.api.common.operators.UnaryOperatorInformation; -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.ReduceOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; - -import static java.util.Arrays.asList; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link ReduceOperator}. */ -@SuppressWarnings({"serial", "unchecked"}) -class ReduceOperatorTest implements Serializable { - - private static final TypeInformation> STRING_INT_TUPLE = - TypeInformation.of(new TypeHint>() {}); - - @Test - void testReduceCollection() { - try { - final ReduceFunction> reducer = - (value1, value2) -> new Tuple2<>(value1.f0, value1.f1 + value2.f1); - - ReduceOperatorBase, ReduceFunction>> - op = - new ReduceOperatorBase<>( - reducer, - new UnaryOperatorInformation<>( - STRING_INT_TUPLE, STRING_INT_TUPLE), - new int[] {0}, - "TestReducer"); - - List> input = - new ArrayList<>( - asList( - new Tuple2<>("foo", 1), - new Tuple2<>("foo", 3), - new Tuple2<>("bar", 2), - new Tuple2<>("bar", 4))); - - ExecutionConfig executionConfig = new ExecutionConfig(); - executionConfig.disableObjectReuse(); - List> resultMutableSafe = - op.executeOnCollections(input, null, executionConfig); - executionConfig.enableObjectReuse(); - List> resultRegular = - op.executeOnCollections(input, null, executionConfig); - - Set> resultSetMutableSafe = new HashSet<>(resultMutableSafe); - Set> resultSetRegular = new HashSet<>(resultRegular); - - Set> expectedResult = - new HashSet<>(asList(new Tuple2<>("foo", 4), new Tuple2<>("bar", 6))); - - assertThat(resultSetMutableSafe).isEqualTo(expectedResult); - assertThat(resultSetRegular).isEqualTo(expectedResult); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testReduceCollectionWithRuntimeContext() { - try { - final String taskName = "Test Task"; - final AtomicBoolean opened = new AtomicBoolean(); - final AtomicBoolean closed = new AtomicBoolean(); - - final ReduceFunction> reducer = - new RichReduceFunction>() { - - @Override - public Tuple2 reduce( - Tuple2 value1, Tuple2 value2) - throws Exception { - - return new Tuple2<>(value1.f0, value1.f1 + value2.f1); - } - - @Override - public void open(OpenContext openContext) throws Exception { - opened.set(true); - RuntimeContext ctx = getRuntimeContext(); - assertThat(ctx.getTaskInfo().getIndexOfThisSubtask()).isZero(); - assertThat(ctx.getTaskInfo().getNumberOfParallelSubtasks()).isOne(); - assertThat(ctx.getTaskInfo().getTaskName()).isEqualTo(taskName); - } - - @Override - public void close() throws Exception { - closed.set(true); - } - }; - - ReduceOperatorBase, ReduceFunction>> - op = - new ReduceOperatorBase<>( - reducer, - new UnaryOperatorInformation<>( - STRING_INT_TUPLE, STRING_INT_TUPLE), - new int[] {0}, - "TestReducer"); - - List> input = - new ArrayList<>( - asList( - new Tuple2<>("foo", 1), - new Tuple2<>("foo", 3), - new Tuple2<>("bar", 2), - new Tuple2<>("bar", 4))); - - final TaskInfo taskInfo = new TaskInfoImpl(taskName, 1, 0, 1, 0); - - ExecutionConfig executionConfig = new ExecutionConfig(); - - executionConfig.disableObjectReuse(); - List> resultMutableSafe = - op.executeOnCollections( - input, - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - new HashMap<>(), - new HashMap<>(), - UnregisteredMetricsGroup.createOperatorMetricGroup()), - executionConfig); - - executionConfig.enableObjectReuse(); - List> resultRegular = - op.executeOnCollections( - input, - new RuntimeUDFContext( - taskInfo, - null, - executionConfig, - new HashMap<>(), - new HashMap<>(), - UnregisteredMetricsGroup.createOperatorMetricGroup()), - executionConfig); - - Set> resultSetMutableSafe = new HashSet<>(resultMutableSafe); - Set> resultSetRegular = new HashSet<>(resultRegular); - - Set> expectedResult = - new HashSet<>(asList(new Tuple2<>("foo", 4), new Tuple2<>("bar", 6))); - - assertThat(resultSetMutableSafe).isEqualTo(expectedResult); - assertThat(resultSetRegular).isEqualTo(expectedResult); - - assertThat(opened.get()).isTrue(); - assertThat(closed.get()).isTrue(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/ExecutionEnvironmentTest.java b/flink-java/src/test/java/org/apache/flink/api/java/ExecutionEnvironmentTest.java deleted file mode 100644 index e659b31a949f5..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/ExecutionEnvironmentTest.java +++ /dev/null @@ -1,71 +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.api.java; - -import org.apache.flink.core.testutils.CheckedThread; -import org.apache.flink.core.testutils.OneShotLatch; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CountDownLatch; - -import static org.assertj.core.api.Assertions.assertThat; - -class ExecutionEnvironmentTest { - - @Test - void testConcurrentSetContext() throws Exception { - int numThreads = 20; - final CountDownLatch waitingThreadCount = new CountDownLatch(numThreads); - final OneShotLatch latch = new OneShotLatch(); - final List threads = new ArrayList<>(); - for (int x = 0; x < numThreads; x++) { - final CheckedThread thread = - new CheckedThread() { - @Override - public void go() { - final ExecutionEnvironment preparedEnvironment = - new ExecutionEnvironment(); - ExecutionEnvironment.initializeContextEnvironment( - () -> preparedEnvironment); - try { - waitingThreadCount.countDown(); - latch.awaitQuietly(); - assertThat(ExecutionEnvironment.getExecutionEnvironment()) - .isSameAs(preparedEnvironment); - } finally { - ExecutionEnvironment.resetContextEnvironment(); - } - } - }; - thread.start(); - threads.add(thread); - } - - // wait for all threads to be ready and trigger the job submissions at the same time - waitingThreadCount.await(); - latch.trigger(); - - for (CheckedThread thread : threads) { - thread.sync(); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/ExecutorDiscoveryAndJobClientTest.java b/flink-java/src/test/java/org/apache/flink/api/java/ExecutorDiscoveryAndJobClientTest.java deleted file mode 100644 index 30dede39dc881..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/ExecutorDiscoveryAndJobClientTest.java +++ /dev/null @@ -1,93 +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.api.java; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.PipelineExecutor; -import org.apache.flink.core.execution.PipelineExecutorFactory; - -import org.junit.jupiter.api.Test; - -import java.util.Collections; -import java.util.concurrent.CompletableFuture; - -import static org.assertj.core.api.Assertions.assertThat; - -/** - * Tests the {@link PipelineExecutorFactory} discovery in the {@link ExecutionEnvironment} and the - * calls of the {@link JobClient}. - */ -class ExecutorDiscoveryAndJobClientTest { - - private static final String EXEC_NAME = "test-executor"; - - @Test - void jobClientGetJobExecutionResultShouldBeCalledOnAttachedExecution() throws Exception { - testHelper(true); - } - - @Test - void jobClientGetJobExecutionResultShouldBeCalledOnDetachedExecution() throws Exception { - testHelper(false); - } - - private void testHelper(final boolean attached) throws Exception { - final Configuration configuration = new Configuration(); - configuration.set(DeploymentOptions.TARGET, EXEC_NAME); - configuration.set(DeploymentOptions.ATTACHED, attached); - - final JobExecutionResult result = executeTestJobBasedOnConfig(configuration); - - assertThat(result.isJobExecutionResult()).isEqualTo(attached); - } - - private JobExecutionResult executeTestJobBasedOnConfig(final Configuration configuration) - throws Exception { - final ExecutionEnvironment env = new ExecutionEnvironment(configuration); - env.fromCollection(Collections.singletonList(42)).output(new DiscardingOutputFormat<>()); - return env.execute(); - } - - /** - * An {@link PipelineExecutorFactory} that returns an {@link PipelineExecutor} that instead of - * executing, it simply returns its name in the {@link JobExecutionResult}. - */ - public static class IDReportingExecutorFactory implements PipelineExecutorFactory { - - @Override - public String getName() { - return EXEC_NAME; - } - - @Override - public boolean isCompatibleWith(Configuration configuration) { - return EXEC_NAME.equals(configuration.get(DeploymentOptions.TARGET)); - } - - @Override - public PipelineExecutor getExecutor(Configuration configuration) { - return (pipeline, executionConfig, classLoader) -> - CompletableFuture.completedFuture(new TestingJobClient()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/MultipleInvokationsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/MultipleInvokationsTest.java deleted file mode 100644 index 4fd4202fbc2ab..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/MultipleInvokationsTest.java +++ /dev/null @@ -1,73 +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.api.java; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.java.io.DiscardingOutputFormat; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for multiple invocations of a plan. */ -class MultipleInvokationsTest { - - @Test - void testMultipleInvocationsGetPlan() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // ----------- Execution 1 --------------- - - DataSet data = env.fromElements("Some", "test", "data").name("source1"); - // data.print(); - data.output(new DiscardingOutputFormat<>()).name("print1"); - data.output(new DiscardingOutputFormat<>()).name("output1"); - - { - Plan p = env.createProgramPlan(); - - assertThat(p.getDataSinks()).hasSize(2); - for (GenericDataSinkBase sink : p.getDataSinks()) { - assertThat(sink.getName()).isIn("print1", "output1"); - assertThat(sink.getInput().getName()).isEqualTo("source1"); - } - } - - // ----------- Execution 2 --------------- - - data.writeAsText("/some/file/path").name("textsink"); - - { - Plan p = env.createProgramPlan(); - - assertThat(p.getDataSinks()).hasSize(1); - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - assertThat(sink.getName()).isEqualTo("textsink"); - assertThat(sink.getInput().getName()).isEqualTo("source1"); - } - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/TestingJobClient.java b/flink-java/src/test/java/org/apache/flink/api/java/TestingJobClient.java deleted file mode 100644 index 989aae9feb6da..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/TestingJobClient.java +++ /dev/null @@ -1,75 +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.api.java; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.SavepointFormatType; - -import javax.annotation.Nullable; - -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -/** Testing implementation of {@link JobClient}. */ -public class TestingJobClient implements JobClient { - - @Override - public JobID getJobID() { - return new JobID(); - } - - @Override - public CompletableFuture getJobStatus() { - return CompletableFuture.completedFuture(JobStatus.FINISHED); - } - - @Override - public CompletableFuture getJobExecutionResult() { - return CompletableFuture.completedFuture( - new JobExecutionResult(new JobID(), 0L, Collections.emptyMap())); - } - - @Override - public CompletableFuture cancel() { - return CompletableFuture.completedFuture(null); - } - - @Override - public CompletableFuture stopWithSavepoint( - boolean advanceToEndOfEventTime, - @Nullable String savepointDirectory, - SavepointFormatType formatType) { - return CompletableFuture.completedFuture("null"); - } - - @Override - public CompletableFuture triggerSavepoint( - @Nullable String savepointDirectory, SavepointFormatType formatType) { - return CompletableFuture.completedFuture("null"); - } - - @Override - public CompletableFuture> getAccumulators() { - return CompletableFuture.completedFuture(Collections.emptyMap()); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/TypeExtractionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/TypeExtractionTest.java deleted file mode 100644 index 32d54652f5587..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/TypeExtractionTest.java +++ /dev/null @@ -1,126 +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.api.java; - -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests concerning type extraction by ExecutionEnvironment methods. */ -@SuppressWarnings("serial") -class TypeExtractionTest { - - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test - void testFunctionWithMissingGenericsAndReturns() { - - RichMapFunction function = - new RichMapFunction() { - private static final long serialVersionUID = 1L; - - @Override - public Object map(Object value) throws Exception { - return null; - } - }; - - TypeInformation info = - ExecutionEnvironment.getExecutionEnvironment() - .fromElements("arbitrary", "data") - .map(function) - .returns(Types.STRING) - .getResultType(); - - assertThat(info).isEqualTo(Types.STRING); - } - - @Test - void testGetterSetterWithVertex() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements(new VertexTyped(0L, 3.0), new VertexTyped(1L, 1.0)); - } - - // ------------------------------------------------------------------------ - // Test types - // ------------------------------------------------------------------------ - - /** - * Representation of Vertex with maximum of 2 keys and a value. - * - * @param keys type - * @param value type - */ - public static class Vertex { - - private K key1; - private K key2; - private V value; - - public Vertex() {} - - public Vertex(K key, V value) { - this.key1 = key; - this.key2 = key; - this.value = value; - } - - public Vertex(K key1, K key2, V value) { - this.key1 = key1; - this.key2 = key2; - this.value = value; - } - - public void setKey1(K key1) { - this.key1 = key1; - } - - public void setKey2(K key2) { - this.key2 = key2; - } - - public K getKey1() { - return key1; - } - - public K getKey2() { - return key2; - } - - public void setValue(V value) { - this.value = value; - } - - public V getValue() { - return value; - } - } - - /** A {@link Vertex} with {@link Long} as key and {@link Double} as value. */ - public static class VertexTyped extends Vertex { - public VertexTyped(Long l, Double d) { - super(l, d); - } - - public VertexTyped() {} - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java deleted file mode 100644 index e091012288192..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SelectByFunctionsTest.java +++ /dev/null @@ -1,234 +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.api.java.functions; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link SelectByMaxFunction} and {@link SelectByMinFunction}. */ -class SelectByFunctionsTest { - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private final Tuple5 bigger = - new Tuple5<>(10, 100L, "HelloWorld", 200L, 20); - private final Tuple5 smaller = - new Tuple5<>(5, 50L, "Hello", 50L, 15); - - // Special case where only the last value determines if bigger or smaller - private final Tuple5 specialCaseBigger = - new Tuple5<>(10, 100L, "HelloWorld", 200L, 17); - private final Tuple5 specialCaseSmaller = - new Tuple5<>(5, 50L, "Hello", 50L, 17); - - /** - * This test validates whether the order of tuples has any impact on the outcome and if the - * bigger tuple is returned. - */ - @Test - void testMaxByComparison() { - SelectByMaxFunction> maxByTuple = - new SelectByMaxFunction<>(tupleTypeInfo, 0); - - try { - assertThat(maxByTuple.reduce(smaller, bigger)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - assertThat(maxByTuple.reduce(bigger, smaller)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - // ----------------------- MAXIMUM FUNCTION TEST BELOW -------------------------- - - /** - * This test cases checks when two tuples only differ in one value, but this value is not in the - * fields list. In that case it should be seen as equal and then the first given tuple (value1) - * should be returned by reduce(). - */ - @Test - void testMaxByComparisonSpecialCase1() { - SelectByMaxFunction> maxByTuple = - new SelectByMaxFunction<>(tupleTypeInfo, 0, 3); - - try { - assertThat(maxByTuple.reduce(specialCaseBigger, bigger)) - .as("SelectByMax must return the first given tuple") - .isSameAs(specialCaseBigger); - assertThat(maxByTuple.reduce(bigger, specialCaseBigger)) - .as("SelectByMax must return the first given tuple") - .isSameAs(bigger); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - /** This test cases checks when two tuples only differ in one value. */ - @Test - void testMaxByComparisonSpecialCase2() { - SelectByMaxFunction> maxByTuple = - new SelectByMaxFunction<>(tupleTypeInfo, 0, 2, 1, 4, 3); - - try { - assertThat(maxByTuple.reduce(specialCaseBigger, bigger)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - assertThat(maxByTuple.reduce(bigger, specialCaseBigger)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - /** This test validates that equality is independent of the amount of used indices. */ - @Test - void testMaxByComparisonMultiple() { - SelectByMaxFunction> maxByTuple = - new SelectByMaxFunction<>(tupleTypeInfo, 0, 1, 2, 3, 4); - - try { - assertThat(maxByTuple.reduce(smaller, bigger)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - assertThat(maxByTuple.reduce(bigger, smaller)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - /** Checks whether reduce does behave as expected if both values are the same object. */ - @Test - void testMaxByComparisonMustReturnATuple() { - SelectByMaxFunction> maxByTuple = - new SelectByMaxFunction<>(tupleTypeInfo, 0); - - try { - assertThat(maxByTuple.reduce(bigger, bigger)) - .as("SelectByMax must return bigger tuple") - .isSameAs(bigger); - assertThat(maxByTuple.reduce(smaller, smaller)) - .as("SelectByMax must return smaller tuple") - .isSameAs(smaller); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - // ----------------------- MINIMUM FUNCTION TEST BELOW -------------------------- - - /** - * This test validates whether the order of tuples has any impact on the outcome and if the - * smaller tuple is returned. - */ - @Test - void testMinByComparison() { - SelectByMinFunction> minByTuple = - new SelectByMinFunction<>(tupleTypeInfo, 0); - - try { - assertThat(minByTuple.reduce(smaller, bigger)) - .as("SelectByMin must return smaller tuple") - .isSameAs(smaller); - assertThat(minByTuple.reduce(bigger, smaller)) - .as("SelectByMin must return smaller tuple") - .isSameAs(smaller); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - /** - * This test cases checks when two tuples only differ in one value, but this value is not in the - * fields list. In that case it should be seen as equal and then the first given tuple (value1) - * should be returned by reduce(). - */ - @Test - void testMinByComparisonSpecialCase1() { - SelectByMinFunction> minByTuple = - new SelectByMinFunction<>(tupleTypeInfo, 0, 3); - - try { - assertThat(minByTuple.reduce(specialCaseBigger, bigger)) - .as("SelectByMin must return the first given tuple") - .isSameAs(specialCaseBigger); - assertThat(minByTuple.reduce(bigger, specialCaseBigger)) - .as("SelectByMin must return the first given tuple") - .isSameAs(bigger); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - /** - * This test validates that when two tuples only differ in one value and that value's index is - * given at construction time. The smaller tuple must be returned then. - */ - @Test - void testMinByComparisonSpecialCase2() { - SelectByMinFunction> minByTuple = - new SelectByMinFunction<>(tupleTypeInfo, 0, 2, 1, 4, 3); - - try { - assertThat(minByTuple.reduce(specialCaseSmaller, smaller)) - .as("SelectByMin must return smaller tuple") - .isSameAs(smaller); - assertThat(minByTuple.reduce(smaller, specialCaseSmaller)) - .as("SelectByMin must return smaller tuple") - .isSameAs(smaller); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } - - /** Checks whether reduce does behave as expected if both values are the same object. */ - @Test - void testMinByComparisonMultiple() { - SelectByMinFunction> minByTuple = - new SelectByMinFunction<>(tupleTypeInfo, 0, 1, 2, 3, 4); - - try { - assertThat(minByTuple.reduce(smaller, bigger)) - .as("SelectByMin must return smaller tuple") - .isSameAs(smaller); - assertThat(minByTuple.reduce(bigger, smaller)) - .as("SelectByMin must return smaller tuple") - .isSameAs(smaller); - } catch (Exception e) { - fail("No exception should be thrown while comparing both tuples"); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java deleted file mode 100644 index 5bf76ef4cd37b..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java +++ /dev/null @@ -1,1839 +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.api.java.functions; - -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties.InvalidSemanticAnnotationException; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.CompositeType; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for semantic properties utils. */ -class SemanticPropUtilTest { - - private final TypeInformation threeIntTupleType = - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private final TypeInformation fourIntTupleType = - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private final TypeInformation fiveIntTupleType = - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private final TypeInformation threeMixedTupleType = - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO); - - private final TypeInformation nestedTupleType = - new TupleTypeInfo, Integer, Integer>>( - threeIntTupleType, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); - - private final TypeInformation deepNestedTupleType = - new TupleTypeInfo< - Tuple3< - Integer, - Tuple3, Integer, Integer>, - Integer>>( - BasicTypeInfo.INT_TYPE_INFO, nestedTupleType, BasicTypeInfo.INT_TYPE_INFO); - - private final TypeInformation pojoType = TypeExtractor.getForClass(TestPojo.class); - - private final TypeInformation pojo2Type = TypeExtractor.getForClass(TestPojo2.class); - - private final TypeInformation nestedPojoType = - TypeExtractor.getForClass(NestedTestPojo.class); - - private final TypeInformation pojoInTupleType = - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, pojoType); - - private final TypeInformation intType = BasicTypeInfo.INT_TYPE_INFO; - - // -------------------------------------------------------------------------------------------- - // Projection Operator Properties - // -------------------------------------------------------------------------------------------- - - @Test - void testSingleProjectionProperties() { - - int[] pMap = new int[] {3, 0, 4}; - SingleInputSemanticProperties sp = - SemanticPropUtil.createProjectionPropertiesSingle( - pMap, (CompositeType) fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(2); - - pMap = new int[] {2, 2, 1, 1}; - sp = - SemanticPropUtil.createProjectionPropertiesSingle( - pMap, (CompositeType) fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 1)).containsExactly(2, 3); - assertThat(sp.getForwardingTargetFields(0, 2)).containsExactly(0, 1); - - pMap = new int[] {2, 0}; - sp = - SemanticPropUtil.createProjectionPropertiesSingle( - pMap, (CompositeType) nestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(3); - - pMap = new int[] {2, 0, 1}; - sp = - SemanticPropUtil.createProjectionPropertiesSingle( - pMap, (CompositeType) deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 6)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(6); - - pMap = new int[] {2, 1}; - sp = - SemanticPropUtil.createProjectionPropertiesSingle( - pMap, (CompositeType) pojoInTupleType); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(4); - } - - @Test - void testDualProjectionProperties() { - - int[] pMap = new int[] {4, 2, 0, 1, 3, 4}; - boolean[] iMap = new boolean[] {true, true, false, true, false, false}; - DualInputSemanticProperties sp = - SemanticPropUtil.createProjectionPropertiesDual( - pMap, iMap, fiveIntTupleType, fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(1, 0)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(3); - assertThat(sp.getForwardingTargetFields(1, 3)).contains(4); - assertThat(sp.getForwardingTargetFields(1, 4)).contains(5); - - pMap = new int[] {4, 2, 0, 4, 0, 1}; - iMap = new boolean[] {true, true, false, true, false, false}; - sp = - SemanticPropUtil.createProjectionPropertiesDual( - pMap, iMap, fiveIntTupleType, fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 4)).containsExactly(0, 3); - assertThat(sp.getForwardingTargetFields(1, 0)).containsExactly(4, 2); - assertThat(sp.getForwardingTargetFields(0, 2)).containsExactly(1); - assertThat(sp.getForwardingTargetFields(1, 1)).containsExactly(5); - - pMap = new int[] {2, 1, 0, 1}; - iMap = new boolean[] {false, false, true, true}; - sp = - SemanticPropUtil.createProjectionPropertiesDual( - pMap, iMap, nestedTupleType, threeIntTupleType); - assertThat(sp.getForwardingTargetFields(1, 2)).contains(0); - assertThat(sp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(5); - - pMap = new int[] {1, 0, 0}; - iMap = new boolean[] {false, false, true}; - sp = - SemanticPropUtil.createProjectionPropertiesDual( - pMap, iMap, nestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(1, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(1, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(1, 3)).contains(2); - assertThat(sp.getForwardingTargetFields(1, 4)).contains(3); - assertThat(sp.getForwardingTargetFields(1, 5)).contains(4); - assertThat(sp.getForwardingTargetFields(1, 0)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(6); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(7); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(8); - - pMap = new int[] {4, 2, 1, 0}; - iMap = new boolean[] {true, false, true, false}; - sp = - SemanticPropUtil.createProjectionPropertiesDual( - pMap, iMap, fiveIntTupleType, pojoInTupleType); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(0); - assertThat(sp.getForwardingTargetFields(1, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(1, 3)).contains(2); - assertThat(sp.getForwardingTargetFields(1, 4)).contains(3); - assertThat(sp.getForwardingTargetFields(1, 5)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(5); - assertThat(sp.getForwardingTargetFields(1, 0)).contains(6); - - pMap = new int[] {2, 3, -1, 0}; - iMap = new boolean[] {true, true, false, true}; - sp = SemanticPropUtil.createProjectionPropertiesDual(pMap, iMap, fiveIntTupleType, intType); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(1); - assertThat(sp.getForwardingTargetFields(1, 0)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(3); - - pMap = new int[] {-1, -1}; - iMap = new boolean[] {false, true}; - sp = SemanticPropUtil.createProjectionPropertiesDual(pMap, iMap, intType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(1, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(1, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(1, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(1, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(1, 5)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(6); - - pMap = new int[] {-1, -1}; - iMap = new boolean[] {true, false}; - sp = SemanticPropUtil.createProjectionPropertiesDual(pMap, iMap, intType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(1, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(1, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(1, 2)).contains(3); - assertThat(sp.getForwardingTargetFields(1, 3)).contains(4); - assertThat(sp.getForwardingTargetFields(1, 4)).contains(5); - assertThat(sp.getForwardingTargetFields(1, 5)).contains(6); - } - - // -------------------------------------------------------------------------------------------- - // Offset - // -------------------------------------------------------------------------------------------- - - @Test - void testAddSourceFieldOffset() { - - SingleInputSemanticProperties semProps = new SingleInputSemanticProperties(); - semProps.addForwardedField(0, 1); - semProps.addForwardedField(0, 4); - semProps.addForwardedField(2, 0); - semProps.addForwardedField(4, 3); - semProps.addReadFields(new FieldSet(0, 3)); - - SemanticProperties offsetProps = SemanticPropUtil.addSourceFieldOffset(semProps, 5, 0); - - assertThat(offsetProps.getForwardingTargetFields(0, 0)).containsExactly(4, 1); - assertThat(offsetProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(offsetProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 4)).containsExactly(3); - - assertThat(offsetProps.getReadFields(0)).containsExactly(0, 3); - - offsetProps = SemanticPropUtil.addSourceFieldOffset(semProps, 5, 3); - - assertThat(offsetProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 3)).containsExactly(4, 1); - assertThat(offsetProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 5)).containsExactly(0); - assertThat(offsetProps.getForwardingTargetFields(0, 6)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 7)).containsExactly(3); - - assertThat(offsetProps.getReadFields(0)).containsExactly(6, 3); - - semProps = new SingleInputSemanticProperties(); - SemanticPropUtil.addSourceFieldOffset(semProps, 1, 0); - - semProps = new SingleInputSemanticProperties(); - semProps.addForwardedField(0, 0); - semProps.addForwardedField(1, 2); - semProps.addForwardedField(2, 4); - - offsetProps = SemanticPropUtil.addSourceFieldOffset(semProps, 3, 2); - - assertThat(offsetProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(offsetProps.getForwardingTargetFields(0, 3)).containsExactly(2); - assertThat(offsetProps.getForwardingTargetFields(0, 4)).containsExactly(4); - } - - @Test - void testAddSourceFieldOffsets() { - - DualInputSemanticProperties semProps = new DualInputSemanticProperties(); - semProps.addForwardedField(0, 0, 1); - semProps.addForwardedField(0, 3, 3); - semProps.addForwardedField(1, 1, 2); - semProps.addForwardedField(1, 1, 4); - semProps.addReadFields(0, new FieldSet(1, 2)); - semProps.addReadFields(1, new FieldSet(0, 3, 4)); - - DualInputSemanticProperties offsetProps = - SemanticPropUtil.addSourceFieldOffsets(semProps, 4, 3, 1, 2); - - assertThat(offsetProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 1)).containsExactly(1); - assertThat(offsetProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(0, 4)).containsExactly(3); - - assertThat(offsetProps.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(1, 2)).isEmpty(); - assertThat(offsetProps.getForwardingTargetFields(1, 3)).containsExactly(4, 2); - - assertThat(offsetProps.getReadFields(0)).containsExactly(2, 3); - assertThat(offsetProps.getReadFields(1)).containsExactly(2, 5, 6); - - semProps = new DualInputSemanticProperties(); - SemanticPropUtil.addSourceFieldOffsets(semProps, 4, 3, 2, 2); - } - - // -------------------------------------------------------------------------------------------- - // Forwarded Fields Annotation - // -------------------------------------------------------------------------------------------- - - @Test - void testForwardedNoArrowIndividualStrings() { - String[] forwardedFields = {"f2", "f3", "f0"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - } - - @Test - void testForwardedNoArrowOneString() { - String[] forwardedFields = {"f2;f3;f0"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - - forwardedFields[0] = "2;3;0"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - - forwardedFields[0] = "2;3;0;"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - } - - @Test - void testForwardedNoArrowSpaces() { - String[] forwardedFields = {" f2 ; f3 ; f0 "}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - } - - @Test - void testForwardedWithArrowIndividualStrings() { - String[] forwardedFields = {"f0->f1", "f1->f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - } - - @Test - void testForwardedWithArrowOneString() { - String[] forwardedFields = {"f0->f0;f1->f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - - forwardedFields[0] = "0->0;1->2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, fiveIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - } - - @Test - void testForwardedWithArrowSpaces() { - String[] forwardedFields = {" f0 -> f0 ; f1 -> f2 "}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - } - - @Test - void testForwardedMixedOneString() { - String[] forwardedFields = {"f2;f3;f0->f4;f4->f0"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, fiveIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(0); - } - - @Test - void testForwardedBasicType() { - String[] forwardedFields = {"f1->*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, intType); - - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - - forwardedFields[0] = "*->f2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, intType, threeIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(2); - - forwardedFields[0] = "*->*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, intType, intType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - } - - @Test - void testForwardedWildCard() { - String[] forwardedFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).isEmpty(); - - forwardedFields[0] = "*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - } - - @Test - void testForwardedNestedTuples() { - String[] forwardedFields = {"f0->f0.f0; f1->f0.f1; f2->f0.f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, nestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - - forwardedFields[0] = "f0.f0->f1.f0.f2; f0.f1->f2; f2->f1.f2; f1->f0"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(6); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(0); - - forwardedFields[0] = "0.0->1.0.2; 0.1->2; 2->1.2; 1->0"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(6); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(0); - - forwardedFields[0] = "f1.f0.*->f0.*; f0->f2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, nestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(4); - - forwardedFields[0] = "1.0.*->0.*; 0->2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, nestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(4); - - forwardedFields[0] = "f1.f0->f0; f0->f2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, nestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(4); - - forwardedFields[0] = "1.0->0; 0->2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, nestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(4); - - forwardedFields[0] = "f1.f0.f1; f1.f1; f2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 6)).contains(6); - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 5)).isEmpty(); - - forwardedFields[0] = "f1.f0.*; f1.f2"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, deepNestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 6)).isEmpty(); - } - - @Test - void testForwardedPojo() { - - String[] forwardedFields = {"int1->int2; int3->int1; string1 "}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, pojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - - forwardedFields[0] = "f1->int1; f0->int3 "; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, threeIntTupleType, pojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - - forwardedFields[0] = "int1->f2; int2->f0; int3->f1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, threeIntTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(1); - - forwardedFields[0] = "*->pojo1.*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(4); - - forwardedFields[0] = "*->pojo1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(4); - - forwardedFields[0] = "int1; string1; int2->pojo1.int3"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(5); - - forwardedFields[0] = "pojo1.*->f2.*; int1->f1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedPojoType, pojoInTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(5); - - forwardedFields[0] = "f2.*->*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoInTupleType, pojoType); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(3); - - forwardedFields[0] = "pojo1->f2; int1->f1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedPojoType, pojoInTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(5); - - forwardedFields[0] = "f2->*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoInTupleType, pojoType); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(3); - - forwardedFields[0] = "int2; string1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, pojoType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - - forwardedFields[0] = "pojo1.int1; string1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedPojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 4)).isEmpty(); - - forwardedFields[0] = "pojo1.*; int1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedPojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 5)).isEmpty(); - - forwardedFields[0] = "pojo1; int1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, nestedPojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 5)).isEmpty(); - } - - @Test - void testInvalidPojoField() { - String[] forwardedFields = {"invalidField"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - pojoType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedNoArrowOneStringInvalidDelimiter() { - String[] forwardedFields = {"f2,f3,f0"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - fiveIntTupleType, - fiveIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedSameTargetTwice() { - String[] forwardedFields = {"f0->f2; f1->f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - fiveIntTupleType, - fiveIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidTargetFieldType1() { - String[] forwardedFields = {"f0->f0", "f1->f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - fiveIntTupleType, - threeMixedTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidTargetFieldType2() { - String[] forwardedFields = {"f2.*->*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - pojoInTupleType, - pojo2Type)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidTargetFieldType3() { - String[] forwardedFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - pojoInTupleType, - pojo2Type)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidTargetFieldType4() { - String[] forwardedFields = {"int1; string1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - pojoInTupleType, - pojo2Type)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidTargetFieldType5() { - String[] forwardedFields = {"f0.*->*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - nestedTupleType, - fiveIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedWildCardInvalidTypes1() { - String[] forwardedFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - fiveIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedWildCardInvalidTypes2() { - String[] forwardedFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - threeIntTupleType, - fiveIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedWildCardInvalidTypes3() { - String[] forwardedFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, pojoType, pojo2Type)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedForwardWildCard() { - String[] forwardedFields = {"f1->*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidExpression() { - String[] forwardedFields = {"f0"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> { - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, null, intType, threeIntTupleType); - }) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedForwardMultiFields() { - String[] forwardedFields = {"f1->f0,f1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedInvalidString() { - String[] forwardedFields = {"notValid"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - forwardedFields, - null, - null, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - // -------------------------------------------------------------------------------------------- - // Non-Forwarded Fields Annotation - // -------------------------------------------------------------------------------------------- - - @Test - void testNonForwardedIndividualStrings() { - String[] nonForwardedFields = {"f1", "f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - } - - @Test - void testNonForwardedSingleString() { - String[] nonForwardedFields = {"f1;f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - - nonForwardedFields[0] = "f1;f2;"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - } - - @Test - void testNonForwardedSpaces() { - String[] nonForwardedFields = {" f1 ; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - } - - @Test - void testNonForwardedNone() { - String[] nonForwardedFields = {""}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - } - - @Test - void testNonForwardedNestedTuple() { - String[] nonForwardedFields = {"f1.f0.*; f1.f2; f0"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, deepNestedTupleType, deepNestedTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 6)).contains(6); - - nonForwardedFields[0] = "f1.f0; f1.f2; f0"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, deepNestedTupleType, deepNestedTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 6)).contains(6); - - nonForwardedFields[0] = "f2; f1.f1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, deepNestedTupleType, deepNestedTupleType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(5); - assertThat(sp.getForwardingTargetFields(0, 6)).isEmpty(); - } - - @Test - void testNonForwardedPojo() { - String[] nonForwardedFields = {"int1; string1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, pojoType, pojoType); - - assertThat(sp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(sp.getForwardingTargetFields(0, 3)).isEmpty(); - } - - @Test - void testNonForwardedNestedPojo() { - String[] nonForwardedFields = {"int1; pojo1.*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, nestedPojoType, nestedPojoType); - - assertThat(sp.getForwardingTargetFields(0, 0).size()).isZero(); - assertThat(sp.getForwardingTargetFields(0, 1).size()).isZero(); - assertThat(sp.getForwardingTargetFields(0, 2).size()).isZero(); - assertThat(sp.getForwardingTargetFields(0, 3).size()).isZero(); - assertThat(sp.getForwardingTargetFields(0, 4).size()).isZero(); - assertThat(sp.getForwardingTargetFields(0, 5)).contains(5); - - nonForwardedFields[0] = "pojo1.int2; string1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, nestedPojoType, nestedPojoType); - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(1); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 3)).contains(3); - assertThat(sp.getForwardingTargetFields(0, 4)).contains(4); - assertThat(sp.getForwardingTargetFields(0, 5)).isEmpty(); - } - - @Test - void testNonForwardedInvalidTypes1() { - String[] nonForwardedFields = {"f1; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - nonForwardedFields, - null, - threeIntTupleType, - nestedPojoType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedInvalidTypes2() { - String[] nonForwardedFields = {"f1; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - nonForwardedFields, - null, - nestedPojoType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedInvalidTypes3() { - String[] nonForwardedFields = {"f1; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - nonForwardedFields, - null, - threeIntTupleType, - fiveIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedInvalidTypes4() { - String[] nonForwardedFields = {"f1; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - nonForwardedFields, - null, - fiveIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedInvalidTypes5() { - String[] nonForwardedFields = {"int1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, null, pojoType, pojo2Type)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedInvalidNesting() { - String[] nonForwardedFields = {"f0.f4"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - nonForwardedFields, - null, - nestedTupleType, - nestedTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedInvalidString() { - String[] nonForwardedFields = {"notValid"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - nonForwardedFields, - null, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - // -------------------------------------------------------------------------------------------- - // Read Fields Annotation - // -------------------------------------------------------------------------------------------- - - @Test - void testReadFieldsIndividualStrings() { - String[] readFields = {"f1", "f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, threeIntTupleType, threeIntTupleType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 2); - } - - @Test - void testReadFieldsOneString() { - String[] readFields = {"f1;f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, threeIntTupleType, threeIntTupleType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 2); - - readFields[0] = "f1;f2;"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, threeIntTupleType, threeIntTupleType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 2); - } - - @Test - void testReadFieldsSpaces() { - String[] readFields = {" f1 ; f2 "}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, threeIntTupleType, threeIntTupleType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).hasSize(2).contains(2, 1); - } - - @Test - void testReadFieldsBasic() { - String[] readFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, intType, intType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(0); - - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, intType, fiveIntTupleType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(0); - } - - @Test - void testReadFieldsNestedTuples() { - String[] readFields = {"f0.f1; f0.f2; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedTupleType, intType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 2, 4); - - readFields[0] = "f0;f1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedTupleType, intType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(0, 1, 2, 3); - } - - @Test - void testReadFieldsNestedTupleWildCard() { - String[] readFields = {"*"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedTupleType, intType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).contains(0, 1, 2, 3, 4); - - readFields[0] = "f0.*;f1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedTupleType, intType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(0, 1, 2, 3); - } - - @Test - void testReadFieldsPojo() { - String[] readFields = {"int2; string1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, pojoType, threeIntTupleType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 3); - - readFields[0] = "*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, pojoType, intType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(0, 1, 2, 3); - } - - @Test - void testReadFieldsNestedPojo() { - String[] readFields = {"pojo1.int2; string1; pojo1.string1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedPojoType, intType); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(2, 4, 5); - - readFields[0] = "pojo1.*"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedPojoType, intType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 2, 3, 4); - - readFields[0] = "pojo1"; - sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, nestedPojoType, intType); - - fs = sp.getReadFields(0); - assertThat(fs).containsExactly(1, 2, 3, 4); - } - - @Test - void testReadFieldsPojoInTuple() { - String[] readFields = {"f0; f2.int1; f2.string1"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, readFields, pojoInTupleType, pojo2Type); - - FieldSet fs = sp.getReadFields(0); - assertThat(fs).containsExactly(0, 2, 5); - } - - @Test - void testReadFieldsInvalidString() { - String[] readFields = {"notValid"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, - null, - null, - readFields, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - // -------------------------------------------------------------------------------------------- - // Two Inputs - // -------------------------------------------------------------------------------------------- - - @Test - void testForwardedDual() { - String[] forwardedFieldsFirst = {"f1->f2; f2->f3"}; - String[] forwardedFieldsSecond = {"f1->f1; f2->f0"}; - DualInputSemanticProperties dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - forwardedFieldsFirst, - forwardedFieldsSecond, - null, - null, - null, - null, - fourIntTupleType, - fourIntTupleType, - fourIntTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(dsp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 2)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 3)).isEmpty(); - - forwardedFieldsFirst[0] = "f1->f0;f3->f1"; - forwardedFieldsSecond[0] = "*->f2.*"; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - forwardedFieldsFirst, - forwardedFieldsSecond, - null, - null, - null, - null, - fourIntTupleType, - pojoType, - pojoInTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 1)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 3)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 0)).contains(2); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(3); - assertThat(dsp.getForwardingTargetFields(1, 2)).contains(4); - assertThat(dsp.getForwardingTargetFields(1, 3)).contains(5); - - forwardedFieldsFirst[0] = "f1.f0.f2->int1; f2->pojo1.int3"; - forwardedFieldsSecond[0] = "string1; int2->pojo1.int1; int1->pojo1.int2"; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - forwardedFieldsFirst, - forwardedFieldsSecond, - null, - null, - null, - null, - deepNestedTupleType, - pojoType, - nestedPojoType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 3)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 6)).contains(3); - assertThat(dsp.getForwardingTargetFields(1, 0)).contains(2); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 3)).contains(5); - - String[] forwardedFieldsFirst2 = {"f1.f0.f2->int1", "f2->pojo1.int3"}; - String[] forwardedFieldsSecond2 = {"string1", "int2->pojo1.int1", "int1->pojo1.int2"}; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - forwardedFieldsFirst2, - forwardedFieldsSecond2, - null, - null, - null, - null, - deepNestedTupleType, - pojoType, - nestedPojoType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 3)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 6)).contains(3); - assertThat(dsp.getForwardingTargetFields(1, 0)).contains(2); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 3)).contains(5); - } - - @Test - void testNonForwardedDual() { - String[] nonForwardedFieldsFirst = {"f1;f2"}; - String[] nonForwardedFieldsSecond = {"f0"}; - DualInputSemanticProperties dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - nonForwardedFieldsFirst, - nonForwardedFieldsSecond, - null, - null, - threeIntTupleType, - threeIntTupleType, - threeIntTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 2)).contains(2); - - nonForwardedFieldsFirst[0] = "f1"; - nonForwardedFieldsSecond[0] = ""; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - nonForwardedFieldsFirst, - null, - null, - null, - threeIntTupleType, - fiveIntTupleType, - threeIntTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(dsp.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 2)).isEmpty(); - - nonForwardedFieldsFirst[0] = ""; - nonForwardedFieldsSecond[0] = "f2;f0"; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - nonForwardedFieldsSecond, - null, - null, - fiveIntTupleType, - threeIntTupleType, - threeIntTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 2)).isEmpty(); - - String[] nonForwardedFields = {"f1", "f3"}; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - nonForwardedFields, - null, - null, - null, - fiveIntTupleType, - threeIntTupleType, - fiveIntTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 2)).contains(2); - assertThat(dsp.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(0, 4)).contains(4); - - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - nonForwardedFields, - null, - null, - threeIntTupleType, - fiveIntTupleType, - fiveIntTupleType); - - assertThat(dsp.getForwardingTargetFields(1, 0)).contains(0); - assertThat(dsp.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 2)).contains(2); - assertThat(dsp.getForwardingTargetFields(1, 3)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 4)).contains(4); - } - - @Test - void testNonForwardedDualInvalidTypes1() { - - String[] nonForwardedFieldsFirst = {"f1"}; - DualInputSemanticProperties dsp = new DualInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - nonForwardedFieldsFirst, - null, - null, - null, - fiveIntTupleType, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testNonForwardedDualInvalidTypes2() { - - String[] nonForwardedFieldsSecond = {"f1"}; - DualInputSemanticProperties dsp = new DualInputSemanticProperties(); - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - nonForwardedFieldsSecond, - null, - null, - threeIntTupleType, - pojoInTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testReadFieldsDual() { - String[] readFieldsFirst = {"f1;f2"}; - String[] readFieldsSecond = {"f0"}; - DualInputSemanticProperties dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - null, - readFieldsFirst, - readFieldsSecond, - threeIntTupleType, - threeIntTupleType, - threeIntTupleType); - - assertThat(dsp.getReadFields(0)).containsExactly(1, 2); - assertThat(dsp.getReadFields(1)).containsExactly(0); - - readFieldsFirst[0] = "f0.*; f2"; - readFieldsSecond[0] = "int1; string1"; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - null, - readFieldsFirst, - readFieldsSecond, - nestedTupleType, - pojoType, - threeIntTupleType); - - assertThat(dsp.getReadFields(0)).containsExactly(0, 1, 2, 4); - assertThat(dsp.getReadFields(1)).containsExactly(0, 3); - - readFieldsFirst[0] = "pojo1.int2; string1"; - readFieldsSecond[0] = "f2.int2"; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - null, - readFieldsFirst, - readFieldsSecond, - nestedPojoType, - pojoInTupleType, - threeIntTupleType); - - assertThat(dsp.getReadFields(0)).hasSize(2).contains(2, 5); - assertThat(dsp.getReadFields(1)).containsExactly(3); - - String[] readFields = {"f0", "f2", "f4"}; - dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - null, - null, - null, - null, - readFields, - readFields, - fiveIntTupleType, - fiveIntTupleType, - threeIntTupleType); - - assertThat(dsp.getReadFields(0)).containsExactly(0, 2, 4); - assertThat(dsp.getReadFields(1)).containsExactly(0, 2, 4); - } - - // -------------------------------------------------------------------------------------------- - // Mixed Annotations - // -------------------------------------------------------------------------------------------- - - @Test - void testForwardedRead() { - String[] forwardedFields = {"f0->f0;f1->f2"}; - String[] readFields = {"f0; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, forwardedFields, null, readFields, threeIntTupleType, fiveIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(sp.getReadFields(0)).containsExactly(0, 2); - } - - @Test - void testNonForwardedRead() { - String[] nonForwardedFields = {"f1;f2"}; - String[] readFields = {"f0; f2"}; - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, nonForwardedFields, readFields, threeIntTupleType, threeIntTupleType); - - assertThat(sp.getForwardingTargetFields(0, 0)).contains(0); - assertThat(sp.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(sp.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(sp.getReadFields(0)).containsExactly(0, 2); - } - - @Test - void testForwardedReadDual() { - String[] forwardedFieldsFirst = {"f1->f2; f2->f3"}; - String[] forwardedFieldsSecond = {"f1->f1; f2->f0"}; - String[] readFieldsFirst = {"0;2"}; - String[] readFieldsSecond = {"1"}; - DualInputSemanticProperties dsp = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dsp, - forwardedFieldsFirst, - forwardedFieldsSecond, - null, - null, - readFieldsFirst, - readFieldsSecond, - fourIntTupleType, - fourIntTupleType, - fourIntTupleType); - - assertThat(dsp.getForwardingTargetFields(0, 1)).contains(2); - assertThat(dsp.getForwardingTargetFields(0, 2)).contains(3); - assertThat(dsp.getForwardingTargetFields(1, 1)).contains(1); - assertThat(dsp.getForwardingTargetFields(1, 2)).contains(0); - assertThat(dsp.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(dsp.getForwardingTargetFields(1, 3)).isEmpty(); - assertThat(dsp.getReadFields(0)).containsExactly(0, 2); - assertThat(dsp.getReadFields(1)).containsExactly(1); - } - - @Test - void testForwardedNonForwardedCheck() { - String[] forwarded = {"1"}; - String[] nonForwarded = {"1"}; - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsSingleFromString( - new SingleInputSemanticProperties(), - forwarded, - nonForwarded, - null, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedNonForwardedFirstCheck() { - String[] forwarded = {"1"}; - String[] nonForwarded = {"1"}; - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsDualFromString( - new DualInputSemanticProperties(), - forwarded, - null, - nonForwarded, - null, - null, - null, - threeIntTupleType, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - @Test - void testForwardedNonForwardedSecondCheck() { - String[] forwarded = {"1"}; - String[] nonForwarded = {"1"}; - assertThatThrownBy( - () -> - SemanticPropUtil.getSemanticPropsDualFromString( - new DualInputSemanticProperties(), - null, - forwarded, - null, - nonForwarded, - null, - null, - threeIntTupleType, - threeIntTupleType, - threeIntTupleType)) - .isInstanceOf(InvalidSemanticAnnotationException.class); - } - - // -------------------------------------------------------------------------------------------- - // Pojo Type Classes - // -------------------------------------------------------------------------------------------- - - /** Sample test pojo. */ - public static class TestPojo { - - public int int1; - public int int2; - public int int3; - public String string1; - } - - /** Sample test pojo. */ - public static class TestPojo2 { - - public int myInt1; - public int myInt2; - public int myInt3; - public String myString1; - } - - /** Sample test pojo with nested type. */ - public static class NestedTestPojo { - - public int int1; - public TestPojo pojo1; - public String string1; - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesPrecedenceTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesPrecedenceTest.java deleted file mode 100644 index 3baf59274a836..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesPrecedenceTest.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.api.java.functions; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -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.Tuple3; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests the precedence of semantic properties: annotation > API. */ -class SemanticPropertiesPrecedenceTest { - - @Test - void testFunctionForwardedAnnotationPrecedence() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(Tuple3.of(3L, "test", 42)); - input.map(new WildcardForwardedMapperWithForwardAnnotation<>()) - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 1); - FieldSet fw3 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(0); - assertThat(fw2).doesNotContain(1); - assertThat(fw3).doesNotContain(2); - } - - @Test - void testFunctionApiPrecedence() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(Tuple3.of(3L, "test", 42)); - input.map(new WildcardForwardedMapper<>()) - .withForwardedFields("f0") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 1); - FieldSet fw3 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(0); - assertThat(fw2).doesNotContain(1); - assertThat(fw3).doesNotContain(2); - } - - // -------------------------------------------------------------------------------------------- - - @FunctionAnnotation.ForwardedFields("f0") - private static class WildcardForwardedMapperWithForwardAnnotation - implements MapFunction { - - @Override - public T map(T value) { - return value; - } - } - - private static class WildcardForwardedMapper implements MapFunction { - - @Override - public T map(T value) { - return value; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java deleted file mode 100644 index baf7e0444cd81..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesProjectionTest.java +++ /dev/null @@ -1,243 +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.api.java.functions; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.CrossOperatorBase; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -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.operators.translation.PlanProjectOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for semantic properties of projected fields. */ -class SemanticPropertiesProjectionTest { - - final List> emptyTupleData = new ArrayList<>(); - - final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - final List, Tuple2, String>> - emptyNestedTupleData = new ArrayList<>(); - - final TupleTypeInfo, Tuple2, String>> - nestedTupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - new TupleTypeInfo>( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO), - new TupleTypeInfo>( - BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO), - BasicTypeInfo.STRING_TYPE_INFO); - - @Test - void testProjectionSemProps1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - tupleDs.project(1, 3, 2, 0, 3).output(new DiscardingOutputFormat<>()); - - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - PlanProjectOperator projectOperator = ((PlanProjectOperator) sink.getInput()); - - SingleInputSemanticProperties props = projectOperator.getSemanticProperties(); - - assertThat(props.getForwardingTargetFields(0, 0)).containsExactly(3); - assertThat(props.getForwardingTargetFields(0, 1)).containsExactly(0); - assertThat(props.getForwardingTargetFields(0, 2)).containsExactly(2); - assertThat(props.getForwardingTargetFields(0, 3)).containsExactly(4, 1); - } - - @Test - void testProjectionSemProps2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet, Tuple2, String>> - tupleDs = env.fromCollection(emptyNestedTupleData, nestedTupleTypeInfo); - - tupleDs.project(2, 3, 1, 2).output(new DiscardingOutputFormat<>()); - - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - PlanProjectOperator projectOperator = ((PlanProjectOperator) sink.getInput()); - - SingleInputSemanticProperties props = projectOperator.getSemanticProperties(); - - assertThat(props.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 1)).containsExactly(3); - assertThat(props.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(props.getForwardingTargetFields(0, 3)).containsExactly(5); - assertThat(props.getForwardingTargetFields(0, 4)).containsExactly(0, 6); - assertThat(props.getForwardingTargetFields(0, 5)).containsExactly(1, 7); - assertThat(props.getForwardingTargetFields(0, 6)).containsExactly(2); - } - - @Test - void testJoinProjectionSemProps1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - tupleDs.join(tupleDs) - .where(0) - .equalTo(0) - .projectFirst(2, 3) - .projectSecond(1, 4) - .output(new DiscardingOutputFormat<>()); - - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase projectJoinOperator = - ((InnerJoinOperatorBase) sink.getInput()); - - DualInputSemanticProperties props = projectJoinOperator.getSemanticProperties(); - - assertThat(props.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(props.getForwardingTargetFields(0, 3)).containsExactly(1); - assertThat(props.getForwardingTargetFields(1, 1)).containsExactly(2); - assertThat(props.getForwardingTargetFields(1, 4)).containsExactly(3); - } - - @Test - void testJoinProjectionSemProps2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet, Tuple2, String>> - tupleDs = env.fromCollection(emptyNestedTupleData, nestedTupleTypeInfo); - - tupleDs.join(tupleDs) - .where(0) - .equalTo(0) - .projectFirst(2, 0) - .projectSecond(1, 3) - .output(new DiscardingOutputFormat<>()); - - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase projectJoinOperator = - ((InnerJoinOperatorBase) sink.getInput()); - - DualInputSemanticProperties props = projectJoinOperator.getSemanticProperties(); - - assertThat(props.getForwardingTargetFields(0, 0)).containsExactly(2); - assertThat(props.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 4)).containsExactly(0); - assertThat(props.getForwardingTargetFields(0, 5)).containsExactly(1); - assertThat(props.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(props.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(props.getForwardingTargetFields(1, 1)).containsExactly(3); - assertThat(props.getForwardingTargetFields(1, 2)).containsExactly(4); - assertThat(props.getForwardingTargetFields(1, 3)).containsExactly(5); - assertThat(props.getForwardingTargetFields(1, 4)).isEmpty(); - assertThat(props.getForwardingTargetFields(1, 5)).isEmpty(); - assertThat(props.getForwardingTargetFields(1, 6)).containsExactly(6); - } - - @Test - void testCrossProjectionSemProps1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - tupleDs.cross(tupleDs) - .projectFirst(2, 3) - .projectSecond(1, 4) - .output(new DiscardingOutputFormat<>()); - - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - CrossOperatorBase projectCrossOperator = - ((CrossOperatorBase) sink.getInput()); - - DualInputSemanticProperties props = projectCrossOperator.getSemanticProperties(); - - assertThat(props.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(props.getForwardingTargetFields(0, 3)).containsExactly(1); - assertThat(props.getForwardingTargetFields(1, 1)).containsExactly(2); - assertThat(props.getForwardingTargetFields(1, 4)).containsExactly(3); - } - - @Test - void testCrossProjectionSemProps2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet, Tuple2, String>> - tupleDs = env.fromCollection(emptyNestedTupleData, nestedTupleTypeInfo); - - tupleDs.cross(tupleDs) - .projectFirst(2, 0) - .projectSecond(1, 3) - .output(new DiscardingOutputFormat<>()); - - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - CrossOperatorBase projectCrossOperator = - ((CrossOperatorBase) sink.getInput()); - - DualInputSemanticProperties props = projectCrossOperator.getSemanticProperties(); - - assertThat(props.getForwardingTargetFields(0, 0)).containsExactly(2); - assertThat(props.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(props.getForwardingTargetFields(0, 4)).containsExactly(0); - assertThat(props.getForwardingTargetFields(0, 5)).containsExactly(1); - assertThat(props.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(props.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(props.getForwardingTargetFields(1, 1)).containsExactly(3); - assertThat(props.getForwardingTargetFields(1, 2)).containsExactly(4); - assertThat(props.getForwardingTargetFields(1, 3)).containsExactly(5); - assertThat(props.getForwardingTargetFields(1, 4)).isEmpty(); - assertThat(props.getForwardingTargetFields(1, 5)).isEmpty(); - assertThat(props.getForwardingTargetFields(1, 6)).containsExactly(6); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java deleted file mode 100644 index 11455bc7dedcb..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropertiesTranslationTest.java +++ /dev/null @@ -1,665 +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.api.java.functions; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -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.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** - * This is a minimal test to verify that semantic annotations are evaluated against the type - * information properly translated correctly to the common data flow API. - */ -@SuppressWarnings("serial") -class SemanticPropertiesTranslationTest { - - @Test - void testUnaryFunctionWildcardForwardedAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = - env.fromElements(new Tuple3<>(3L, "test", 42)); - input.map(new WildcardForwardedMapper<>()).output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 1); - FieldSet fw3 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(0); - assertThat(fw2).contains(1); - assertThat(fw3).contains(2); - } - - @Test - void testUnaryFunctionInPlaceForwardedAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = - env.fromElements(new Tuple3<>(3L, "test", 42)); - input.map(new IndividualForwardedMapper<>()).output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(0); - assertThat(fw2).contains(2); - } - - @Test - void testUnaryFunctionMovingForwardedAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input.map(new ShufflingMapper<>()).output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 1); - FieldSet fw3 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(2); - assertThat(fw2).contains(0); - assertThat(fw3).contains(1); - } - - @Test - void testUnaryFunctionForwardedInLine1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input.map(new NoAnnotationMapper<>()) - .withForwardedFields("0->1; 2") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(1); - assertThat(fw2).contains(2); - } - - @Test - void testUnaryFunctionForwardedInLine2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input.map(new ReadSetMapper<>()) - .withForwardedFields("0->1; 2") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(1); - assertThat(fw2).contains(2); - } - - @Test - void testUnaryFunctionForwardedInLine3() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input.map(new ReadSetMapper<>()) - .withForwardedFields("0->1; 2") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(1); - assertThat(fw2).contains(2); - } - - @Test - void testUnaryFunctionAllForwardedExceptAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input.map(new AllForwardedExceptMapper<>()).output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet fw1 = semantics.getForwardingTargetFields(0, 0); - FieldSet fw2 = semantics.getForwardingTargetFields(0, 2); - assertThat(fw1).contains(0); - assertThat(fw2).contains(2); - } - - @Test - void testUnaryFunctionReadFieldsAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input.map(new ReadSetMapper<>()).output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - MapOperatorBase mapper = (MapOperatorBase) sink.getInput(); - - SingleInputSemanticProperties semantics = mapper.getSemanticProperties(); - - FieldSet read = semantics.getReadFields(0); - assertThat(read).containsExactly(0, 2); - } - - @Test - void testUnaryForwardedOverwritingInLine1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - assertThatThrownBy( - () -> - input.map(new WildcardForwardedMapper<>()) - .withForwardedFields("0->1; 2")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testUnaryForwardedOverwritingInLine2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - assertThatThrownBy( - () -> - input.map(new AllForwardedExceptMapper<>()) - .withForwardedFields("0->1; 2")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testBinaryForwardedAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, "test")); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 3.1415)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedBothAnnotationJoin<>()) - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 1)).containsExactly(0); - assertThat(semantics.getForwardingTargetFields(1, 1)).containsExactly(1); - } - - @Test - void testBinaryForwardedInLine1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new NoAnnotationJoin<>()) - .withForwardedFieldsFirst("0->1; 1->2") - .withForwardedFieldsSecond("1->0") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 0)).containsExactly(1); - assertThat(semantics.getForwardingTargetFields(0, 1)).containsExactly(2); - assertThat(semantics.getForwardingTargetFields(1, 1)).containsExactly(0); - } - - @Test - void testBinaryForwardedInLine2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ReadSetJoin<>()) - .withForwardedFieldsFirst("0->1; 1->2") - .withForwardedFieldsSecond("1->0") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 0)).containsExactly(1); - assertThat(semantics.getForwardingTargetFields(0, 1)).containsExactly(2); - assertThat(semantics.getForwardingTargetFields(1, 1)).containsExactly(0); - assertThat(semantics.getReadFields(0)).containsExactly(1); - assertThat(semantics.getReadFields(1)).containsExactly(0); - } - - @Test - void testBinaryForwardedAnnotationInLineMixed1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedFirstAnnotationJoin<>()) - .withForwardedFieldsSecond("1") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 0)).containsExactly(2); - assertThat(semantics.getForwardingTargetFields(1, 1)).containsExactly(1); - } - - @Test - void testBinaryForwardedAnnotationInLineMixed2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedSecondAnnotationJoin<>()) - .withForwardedFieldsFirst("0->1") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 0)).containsExactly(1); - assertThat(semantics.getForwardingTargetFields(1, 1)).containsExactly(2); - } - - @Test - void testBinaryAllForwardedExceptAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple3<>(3L, 4L, 5L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new AllForwardedExceptJoin<>()) - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(semantics.getForwardingTargetFields(0, 1)).containsExactly(1); - assertThat(semantics.getForwardingTargetFields(1, 2)).containsExactly(2); - } - - @Test - void testBinaryReadFieldsAnnotation() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ReadSetJoin<>()) - .output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - - GenericDataSinkBase sink = plan.getDataSinks().iterator().next(); - InnerJoinOperatorBase join = - (InnerJoinOperatorBase) sink.getInput(); - - DualInputSemanticProperties semantics = join.getSemanticProperties(); - assertThat(semantics.getReadFields(0)).containsExactly(1); - assertThat(semantics.getReadFields(1)).containsExactly(0); - } - - @Test - void testBinaryForwardedOverwritingInLine1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - assertThatThrownBy( - () -> - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedFirstAnnotationJoin<>()) - .withForwardedFieldsFirst("0->1")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testBinaryForwardedOverwritingInLine2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - assertThatThrownBy( - () -> - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedSecondAnnotationJoin<>()) - .withForwardedFieldsSecond("0->1")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testBinaryForwardedOverwritingInLine3() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - assertThatThrownBy( - () -> - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedBothAnnotationJoin<>()) - .withForwardedFieldsFirst("0->1;")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testBinaryForwardedOverwritingInLine4() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple2<>(3L, 4L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple2<>(3L, 2L)); - assertThatThrownBy( - () -> - input1.join(input2) - .where(0) - .equalTo(0) - .with(new ForwardedBothAnnotationJoin<>()) - .withForwardedFieldsSecond("0->1;")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testBinaryForwardedOverwritingInLine5() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple3<>(3L, 4L, 5L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - assertThatThrownBy( - () -> - input1.join(input2) - .where(0) - .equalTo(0) - .with(new AllForwardedExceptJoin<>()) - .withForwardedFieldsFirst("0->1;")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - @Test - void testBinaryForwardedOverwritingInLine6() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input1 = env.fromElements(new Tuple3<>(3L, 4L, 5L)); - @SuppressWarnings("unchecked") - DataSet> input2 = env.fromElements(new Tuple3<>(3L, 2L, 1L)); - assertThatThrownBy( - () -> - input1.join(input2) - .where(0) - .equalTo(0) - .with(new AllForwardedExceptJoin<>()) - .withForwardedFieldsSecond("0->1;")) - .isInstanceOf(SemanticProperties.InvalidSemanticAnnotationException.class); - } - - // -------------------------------------------------------------------------------------------- - - private static class NoAnnotationMapper implements MapFunction { - - @Override - public T map(T value) { - return value; - } - } - - @ForwardedFields("*") - private static class WildcardForwardedMapper implements MapFunction { - - @Override - public T map(T value) { - return value; - } - } - - @ForwardedFields("0;2") - private static class IndividualForwardedMapper - implements MapFunction, Tuple3> { - - @Override - public Tuple3 map(Tuple3 value) { - return value; - } - } - - @ForwardedFields("0->2;1->0;2->1") - private static class ShufflingMapper - implements MapFunction, Tuple3> { - - @Override - public Tuple3 map(Tuple3 value) { - return value; - } - } - - @FunctionAnnotation.NonForwardedFields({"1"}) - private static class AllForwardedExceptMapper implements MapFunction { - - @Override - public T map(T value) { - return value; - } - } - - @FunctionAnnotation.ReadFields({"0;2"}) - private static class ReadSetMapper implements MapFunction { - - @Override - public T map(T value) { - return value; - } - } - - private static class NoAnnotationJoin - implements JoinFunction, Tuple2, Tuple3> { - - @Override - public Tuple3 join(Tuple2 first, Tuple2 second) throws Exception { - return null; - } - } - - @ForwardedFieldsFirst("0->2") - private static class ForwardedFirstAnnotationJoin - implements JoinFunction, Tuple2, Tuple3> { - - @Override - public Tuple3 join(Tuple2 first, Tuple2 second) throws Exception { - return null; - } - } - - @ForwardedFieldsSecond("1->2") - private static class ForwardedSecondAnnotationJoin - implements JoinFunction, Tuple2, Tuple3> { - - @Override - public Tuple3 join(Tuple2 first, Tuple2 second) throws Exception { - return null; - } - } - - @ForwardedFieldsFirst("1 -> 0") - @ForwardedFieldsSecond("1 -> 1") - private static class ForwardedBothAnnotationJoin - implements JoinFunction, Tuple2, Tuple2> { - - @Override - public Tuple2 join(Tuple2 first, Tuple2 second) { - return new Tuple2(first.f1, second.f1); - } - } - - @FunctionAnnotation.NonForwardedFieldsFirst("0;2") - @FunctionAnnotation.NonForwardedFieldsSecond("0;1") - private static class AllForwardedExceptJoin - implements JoinFunction, Tuple3, Tuple3> { - - @Override - public Tuple3 join(Tuple3 first, Tuple3 second) - throws Exception { - return null; - } - } - - @FunctionAnnotation.ReadFieldsFirst("1") - @FunctionAnnotation.ReadFieldsSecond("0") - private static class ReadSetJoin - implements JoinFunction, Tuple2, Tuple3> { - - @Override - public Tuple3 join(Tuple2 first, Tuple2 second) throws Exception { - return null; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java deleted file mode 100644 index a0531b9505e46..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/CSVReaderTest.java +++ /dev/null @@ -1,348 +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.api.java.io; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.BooleanValue; -import org.apache.flink.types.ByteValue; -import org.apache.flink.types.CharValue; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.ShortValue; -import org.apache.flink.types.StringValue; -import org.apache.flink.types.Value; - -import org.junit.jupiter.api.Test; - -import java.io.IOException; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for the CSV reader builder. */ -class CSVReaderTest { - - @Test - void testIgnoreHeaderConfigure() { - CsvReader reader = getCsvReader(); - reader.ignoreFirstLine(); - assertThat(reader.skipFirstLineAsHeader).isTrue(); - } - - @Test - void testIgnoreInvalidLinesConfigure() { - CsvReader reader = getCsvReader(); - assertThat(reader.ignoreInvalidLines).isFalse(); - reader.ignoreInvalidLines(); - assertThat(reader.ignoreInvalidLines).isTrue(); - } - - @Test - void testIgnoreComments() { - CsvReader reader = getCsvReader(); - assertThat(reader.commentPrefix).isNull(); - reader.ignoreComments("#"); - assertThat(reader.commentPrefix).isEqualTo("#"); - } - - @Test - void testCharset() { - CsvReader reader = getCsvReader(); - assertThat(reader.getCharset()).isEqualTo("UTF-8"); - reader.setCharset("US-ASCII"); - assertThat(reader.getCharset()).isEqualTo("US-ASCII"); - } - - @Test - void testIncludeFieldsDense() { - CsvReader reader = getCsvReader(); - reader.includeFields(true, true, true); - assertThat(reader.includedMask).containsExactly(true, true, true); - - reader = getCsvReader(); - reader.includeFields("ttt"); - assertThat(reader.includedMask).containsExactly(true, true, true); - - reader = getCsvReader(); - reader.includeFields("TTT"); - assertThat(reader.includedMask).containsExactly(true, true, true); - - reader = getCsvReader(); - reader.includeFields("111"); - assertThat(reader.includedMask).containsExactly(true, true, true); - - reader = getCsvReader(); - reader.includeFields(0x7L); - assertThat(reader.includedMask).containsExactly(true, true, true); - } - - @Test - void testIncludeFieldsSparse() { - CsvReader reader = getCsvReader(); - reader.includeFields(false, true, true, false, false, true, false, false); - assertThat(reader.includedMask).containsExactly(false, true, true, false, false, true); - - reader = getCsvReader(); - reader.includeFields("fttfftff"); - assertThat(reader.includedMask).containsExactly(false, true, true, false, false, true); - - reader = getCsvReader(); - reader.includeFields("FTTFFTFF"); - assertThat(reader.includedMask).containsExactly(false, true, true, false, false, true); - - reader = getCsvReader(); - reader.includeFields("01100100"); - assertThat(reader.includedMask).containsExactly(false, true, true, false, false, true); - - reader = getCsvReader(); - reader.includeFields("0t1f0TFF"); - assertThat(reader.includedMask).containsExactly(false, true, true, false, false, true); - - reader = getCsvReader(); - reader.includeFields(0x26L); - assertThat(reader.includedMask).containsExactly(false, true, true, false, false, true); - } - - @Test - void testIllegalCharInStringMask() { - CsvReader reader = getCsvReader(); - assertThatThrownBy(() -> reader.includeFields("1t0Tfht")) - .withFailMessage("Reader accepted an invalid mask string") - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testIncludeFieldsErrorWhenExcludingAll() { - CsvReader reader = getCsvReader(); - - assertThatThrownBy(() -> reader.includeFields(false, false, false, false, false, false)) - .withFailMessage( - "The reader accepted a fields configuration that excludes all fields.") - .isInstanceOf(IllegalArgumentException.class); - - assertThatThrownBy(() -> reader.includeFields(0)) - .withFailMessage( - "The reader accepted a fields configuration that excludes all fields.") - .isInstanceOf(IllegalArgumentException.class); - - assertThatThrownBy(() -> reader.includeFields("ffffffffffffff")) - .withFailMessage( - "The reader accepted a fields configuration that excludes all fields.") - .isInstanceOf(IllegalArgumentException.class); - - assertThatThrownBy(() -> reader.includeFields("00000000000000000")) - .withFailMessage( - "The reader accepted a fields configuration that excludes all fields.") - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testReturnType() { - CsvReader reader = getCsvReader(); - DataSource items = reader.tupleType(Item.class); - assertThat(items.getType().getTypeClass()).isSameAs(Item.class); - } - - @Test - void testFieldTypes() { - CsvReader reader = getCsvReader(); - DataSource items = reader.tupleType(Item.class); - - TypeInformation info = items.getType(); - if (!info.isTupleType()) { - fail(""); - } else { - TupleTypeInfo tinfo = (TupleTypeInfo) info; - assertThat(tinfo.getTypeAt(0)).isEqualTo(BasicTypeInfo.INT_TYPE_INFO); - assertThat(tinfo.getTypeAt(1)).isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); - assertThat(tinfo.getTypeAt(2)).isEqualTo(BasicTypeInfo.DOUBLE_TYPE_INFO); - assertThat(tinfo.getTypeAt(3)).isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); - } - - CsvInputFormat inputFormat = (CsvInputFormat) items.getInputFormat(); - assertThat(inputFormat.getFieldTypes()) - .containsExactly(Integer.class, String.class, Double.class, String.class); - } - - @Test - void testSubClass() { - CsvReader reader = getCsvReader(); - DataSource sitems = reader.tupleType(SubItem.class); - TypeInformation info = sitems.getType(); - - assertThat(info.isTupleType()).isTrue(); - assertThat(info.getTypeClass()).isEqualTo(SubItem.class); - - @SuppressWarnings("unchecked") - TupleTypeInfo tinfo = (TupleTypeInfo) info; - - assertThat(tinfo.getTypeAt(0)).isEqualTo(BasicTypeInfo.INT_TYPE_INFO); - assertThat(tinfo.getTypeAt(1)).isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); - assertThat(tinfo.getTypeAt(2)).isEqualTo(BasicTypeInfo.DOUBLE_TYPE_INFO); - assertThat(tinfo.getTypeAt(3)).isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); - - CsvInputFormat inputFormat = (CsvInputFormat) sitems.getInputFormat(); - assertThat(inputFormat.getFieldTypes()) - .containsExactly(Integer.class, String.class, Double.class, String.class); - } - - @Test - void testSubClassWithPartialsInHierarchie() { - CsvReader reader = getCsvReader(); - DataSource sitems = reader.tupleType(FinalItem.class); - TypeInformation info = sitems.getType(); - - assertThat(info.isTupleType()).isTrue(); - assertThat(info.getTypeClass()).isEqualTo(FinalItem.class); - - @SuppressWarnings("unchecked") - TupleTypeInfo tinfo = (TupleTypeInfo) info; - - assertThat(tinfo.getTypeAt(0)).isEqualTo(BasicTypeInfo.INT_TYPE_INFO); - assertThat(tinfo.getTypeAt(1)).isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); - assertThat(tinfo.getTypeAt(2)).isEqualTo(BasicTypeInfo.DOUBLE_TYPE_INFO); - assertThat(tinfo.getTypeAt(3).getClass()).isEqualTo(ValueTypeInfo.class); - assertThat(tinfo.getTypeAt(4).getClass()).isEqualTo(ValueTypeInfo.class); - assertThat((tinfo.getTypeAt(3)).getTypeClass()).isEqualTo(StringValue.class); - assertThat((tinfo.getTypeAt(4)).getTypeClass()).isEqualTo(LongValue.class); - - CsvInputFormat inputFormat = (CsvInputFormat) sitems.getInputFormat(); - assertThat(inputFormat.getFieldTypes()) - .containsExactly( - Integer.class, - String.class, - Double.class, - StringValue.class, - LongValue.class); - } - - @Test - void testUnsupportedPartialitem() { - CsvReader reader = getCsvReader(); - - assertThatThrownBy(() -> reader.tupleType(PartialItem.class)) - .withFailMessage("tupleType() accepted an underspecified generic class.") - .isInstanceOf(Exception.class); - } - - @Test - void testWithValueType() { - CsvReader reader = getCsvReader(); - DataSource< - Tuple8< - StringValue, - BooleanValue, - ByteValue, - ShortValue, - IntValue, - LongValue, - FloatValue, - DoubleValue>> - items = - reader.types( - StringValue.class, - BooleanValue.class, - ByteValue.class, - ShortValue.class, - IntValue.class, - LongValue.class, - FloatValue.class, - DoubleValue.class); - TypeInformation info = items.getType(); - - assertThat(info.isTupleType()).isTrue(); - assertThat(info.getTypeClass()).isEqualTo(Tuple8.class); - } - - @Test - void testWithInvalidValueType1() { - CsvReader reader = getCsvReader(); - // CsvReader doesn't support CharValue - assertThatThrownBy(() -> reader.types(CharValue.class)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testWithInvalidValueType2() { - CsvReader reader = getCsvReader(); - // CsvReader doesn't support custom Value type - assertThatThrownBy(() -> reader.types(ValueItem.class)) - .isInstanceOf(IllegalArgumentException.class); - } - - private static CsvReader getCsvReader() { - return new CsvReader( - "/some/none/existing/path", ExecutionEnvironment.createLocalEnvironment(1)); - } - - // -------------------------------------------------------------------------------------------- - // Custom types for testing - // -------------------------------------------------------------------------------------------- - - private static class Item extends Tuple4 { - private static final long serialVersionUID = -7444437337392053502L; - } - - private static class SubItem extends Item { - private static final long serialVersionUID = 1L; - } - - private static class PartialItem extends Tuple5 { - private static final long serialVersionUID = 1L; - } - - private static class FinalItem extends PartialItem { - private static final long serialVersionUID = 1L; - } - - private static class ValueItem implements Value { - private int v1; - - public int getV1() { - return v1; - } - - public void setV1(int v1) { - this.v1 = v1; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.writeInt(v1); - } - - @Override - public void read(DataInputView in) throws IOException { - v1 = in.readInt(); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java deleted file mode 100644 index e52b8694b93d3..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java +++ /dev/null @@ -1,389 +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.api.java.io; - -import org.apache.flink.api.common.serialization.SerializerConfigImpl; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import org.junit.jupiter.api.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CollectionInputFormat}. */ -class CollectionInputFormatTest { - - private static class ElementType { - private final int id; - - public ElementType() { - this(-1); - } - - public ElementType(int id) { - this.id = id; - } - - public int getId() { - return id; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof ElementType) { - ElementType et = (ElementType) obj; - return et.getId() == this.getId(); - } else { - return false; - } - } - - @Override - public int hashCode() { - return id; - } - - @Override - public String toString() { - return "ElementType{" + "id=" + id + '}'; - } - } - - @Test - void testSerializability() { - try (ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buffer)) { - - Collection inputCollection = new ArrayList<>(); - ElementType element1 = new ElementType(1); - ElementType element2 = new ElementType(2); - ElementType element3 = new ElementType(3); - inputCollection.add(element1); - inputCollection.add(element2); - inputCollection.add(element3); - - @SuppressWarnings("unchecked") - TypeInformation info = TypeExtractor.createTypeInfo(ElementType.class); - - CollectionInputFormat inputFormat = - new CollectionInputFormat<>( - inputCollection, info.createSerializer(new SerializerConfigImpl())); - - out.writeObject(inputFormat); - - ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(buffer.toByteArray())); - - Object serializationResult = in.readObject(); - - assertThat(serializationResult).isInstanceOf(CollectionInputFormat.class); - - @SuppressWarnings("unchecked") - CollectionInputFormat result = - (CollectionInputFormat) serializationResult; - - GenericInputSplit inputSplit = new GenericInputSplit(0, 1); - inputFormat.open(inputSplit); - result.open(inputSplit); - - while (!inputFormat.reachedEnd() && !result.reachedEnd()) { - ElementType expectedElement = inputFormat.nextRecord(null); - ElementType actualElement = result.nextRecord(null); - - assertThat(actualElement).isEqualTo(expectedElement); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.toString()); - } - } - - @Test - void testSerializabilityStrings() { - - final String[] data = - new String[] { - "To be, or not to be,--that is the question:--", - "Whether 'tis nobler in the mind to suffer", - "The slings and arrows of outrageous fortune", - "Or to take arms against a sea of troubles,", - "And by opposing end them?--To die,--to sleep,--", - "No more; and by a sleep to say we end", - "The heartache, and the thousand natural shocks", - "That flesh is heir to,--'tis a consummation", - "Devoutly to be wish'd. To die,--to sleep;--", - "To sleep! perchance to dream:--ay, there's the rub;", - "For in that sleep of death what dreams may come,", - "When we have shuffled off this mortal coil,", - "Must give us pause: there's the respect", - "That makes calamity of so long life;", - "For who would bear the whips and scorns of time,", - "The oppressor's wrong, the proud man's contumely,", - "The pangs of despis'd love, the law's delay,", - "The insolence of office, and the spurns", - "That patient merit of the unworthy takes,", - "When he himself might his quietus make", - "With a bare bodkin? who would these fardels bear,", - "To grunt and sweat under a weary life,", - "But that the dread of something after death,--", - "The undiscover'd country, from whose bourn", - "No traveller returns,--puzzles the will,", - "And makes us rather bear those ills we have", - "Than fly to others that we know not of?", - "Thus conscience does make cowards of us all;", - "And thus the native hue of resolution", - "Is sicklied o'er with the pale cast of thought;", - "And enterprises of great pith and moment,", - "With this regard, their currents turn awry,", - "And lose the name of action.--Soft you now!", - "The fair Ophelia!--Nymph, in thy orisons", - "Be all my sins remember'd." - }; - - try { - List inputCollection = Arrays.asList(data); - CollectionInputFormat inputFormat = - new CollectionInputFormat<>( - inputCollection, - BasicTypeInfo.STRING_TYPE_INFO.createSerializer( - new SerializerConfigImpl())); - - // serialize - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos); - oos.writeObject(inputFormat); - oos.close(); - - // deserialize - ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); - ObjectInputStream ois = new ObjectInputStream(bais); - Object result = ois.readObject(); - - assertThat(result).isInstanceOf(CollectionInputFormat.class); - - int i = 0; - @SuppressWarnings("unchecked") - CollectionInputFormat in = (CollectionInputFormat) result; - in.open(new GenericInputSplit(0, 1)); - - while (!in.reachedEnd()) { - assertThat(in.nextRecord("")).isEqualTo(data[i++]); - } - - assertThat(i).isEqualTo(data.length); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testSerializationFailure() { - try (ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buffer)) { - // a mock serializer that fails when writing - CollectionInputFormat inFormat = - new CollectionInputFormat<>( - Collections.singleton(new ElementType()), - new TestSerializer(false, true)); - - try { - out.writeObject(inFormat); - fail("should throw an exception"); - } catch (TestException e) { - // expected - } catch (Exception e) { - fail("Exception not properly forwarded"); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testDeserializationFailure() { - try (ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buffer)) { - // a mock serializer that fails when writing - CollectionInputFormat inFormat = - new CollectionInputFormat<>( - Collections.singleton(new ElementType()), - new TestSerializer(true, false)); - - out.writeObject(inFormat); - out.close(); - - ByteArrayInputStream bais = new ByteArrayInputStream(buffer.toByteArray()); - ObjectInputStream in = new ObjectInputStream(bais); - - try { - in.readObject(); - fail("should throw an exception"); - } catch (Exception e) { - assertThat(e).hasCauseInstanceOf(TestException.class); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testToStringOnSmallCollection() { - ArrayList smallList = new ArrayList<>(); - smallList.add(new ElementType(1)); - smallList.add(new ElementType(2)); - CollectionInputFormat inputFormat = - new CollectionInputFormat<>(smallList, new TestSerializer(true, false)); - - assertThat(inputFormat).hasToString("[ElementType{id=1}, ElementType{id=2}]"); - } - - @Test - void testToStringOnBigCollection() { - ArrayList list = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - list.add(new ElementType(i)); - } - CollectionInputFormat inputFormat = - new CollectionInputFormat<>(list, new TestSerializer(true, false)); - - assertThat(inputFormat) - .hasToString( - "[ElementType{id=0}, ElementType{id=1}, ElementType{id=2}, " - + "ElementType{id=3}, ElementType{id=4}, ElementType{id=5}, ...]"); - } - - private static class TestException extends IOException { - private static final long serialVersionUID = 1L; - } - - private static class TestSerializer extends TypeSerializer { - - private static final long serialVersionUID = 1L; - - private final boolean failOnRead; - private final boolean failOnWrite; - - public TestSerializer(boolean failOnRead, boolean failOnWrite) { - this.failOnRead = failOnRead; - this.failOnWrite = failOnWrite; - } - - @Override - public boolean isImmutableType() { - return true; - } - - @Override - public TestSerializer duplicate() { - return this; - } - - @Override - public ElementType createInstance() { - return new ElementType(); - } - - @Override - public ElementType copy(ElementType from) { - return from; - } - - @Override - public ElementType copy(ElementType from, ElementType reuse) { - return from; - } - - @Override - public int getLength() { - return 4; - } - - @Override - public void serialize(ElementType record, DataOutputView target) throws IOException { - if (failOnWrite) { - throw new TestException(); - } - target.writeInt(record.getId()); - } - - @Override - public ElementType deserialize(DataInputView source) throws IOException { - if (failOnRead) { - throw new TestException(); - } - return new ElementType(source.readInt()); - } - - @Override - public ElementType deserialize(ElementType reuse, DataInputView source) throws IOException { - if (failOnRead) { - throw new TestException(); - } - return new ElementType(source.readInt()); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - target.writeInt(source.readInt()); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof TestSerializer) { - TestSerializer other = (TestSerializer) obj; - - return failOnRead == other.failOnRead && failOnWrite == other.failOnWrite; - } else { - return false; - } - } - - @Override - public int hashCode() { - return Objects.hash(failOnRead, failOnWrite); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - throw new UnsupportedOperationException(); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java deleted file mode 100644 index 6ea30ab5f2f0f..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java +++ /dev/null @@ -1,1462 +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.api.java.io; - -import org.apache.flink.api.common.io.ParseException; -import org.apache.flink.api.common.io.compression.InflaterInputStreamFactory; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple6; -import org.apache.flink.api.java.typeutils.PojoTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.types.parser.FieldParser; -import org.apache.flink.types.parser.StringParser; - -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStreamWriter; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CsvInputFormat}. */ -class CsvInputFormatTest { - - private static final Path PATH = new Path("an/ignored/file/"); - - // Static variables for testing the removal of \r\n to \n - private static final String FIRST_PART = "That is the first part"; - - private static final String SECOND_PART = "That is the second part"; - - @Test - void testSplitCsvInputStreamInLargeBuffer() throws Exception { - testSplitCsvInputStream(1024 * 1024, false, false); - testSplitCsvInputStream(1024 * 1024, false, true); - } - - @Test - void testSplitCsvInputStreamInSmallBuffer() throws Exception { - testSplitCsvInputStream(2, false, false); - testSplitCsvInputStream(1024 * 1024, false, true); - } - - private void testSplitCsvInputStream(int bufferSize, boolean failAtStart, boolean compressed) - throws Exception { - final String fileContent = - "this is|1|2.0|\n" + "a test|3|4.0|\n" + "#next|5|6.0|\n" + "asdadas|5|30.0|\n"; - - // create temporary file with 3 blocks - final File tempFile; - - if (compressed) { - tempFile = File.createTempFile("TextInputFormatTest", ".compressed"); - TextInputFormat.registerInflaterInputStreamFactory( - "compressed", - new InflaterInputStreamFactory() { - @Override - public InputStream create(InputStream in) { - return in; - } - - @Override - public Collection getCommonFileExtensions() { - return Collections.singletonList("compressed"); - } - }); - } else { - tempFile = File.createTempFile("input-stream-decoration-test", ".tmp"); - } - tempFile.deleteOnExit(); - - try (FileOutputStream fileOutputStream = new FileOutputStream(tempFile)) { - fileOutputStream.write(fileContent.getBytes(ConfigConstants.DEFAULT_CHARSET)); - } - - // fix the number of blocks and the size of each one. - final int noOfBlocks = 3; - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class, Double.class); - CsvInputFormat> format = - new TupleCsvInputFormat<>(new Path(tempFile.toURI()), "\n", "|", typeInfo); - format.setLenient(true); - format.setBufferSize(bufferSize); - - final Configuration config = new Configuration(); - format.configure(config); - - long[] offsetsAfterRecord = new long[] {15, 29, 42, 58}; - long[] offsetAtEndOfSplit = new long[] {20, 40, 58}; - int recordCounter = 0; - int splitCounter = 0; - - FileInputSplit[] inputSplits = format.createInputSplits(noOfBlocks); - Tuple3 result = new Tuple3<>(); - - for (FileInputSplit inputSplit : inputSplits) { - if (!compressed) { - assertThat(offsetAtEndOfSplit[splitCounter]) - .isEqualTo(inputSplit.getStart() + inputSplit.getLength()); - } - splitCounter++; - - format.open(inputSplit); - format.reopen(inputSplit, format.getCurrentState()); - - while (!format.reachedEnd()) { - if ((result = format.nextRecord(result)) != null) { - assertThat(offsetsAfterRecord[recordCounter]) - .isEqualTo((long) format.getCurrentState()); - recordCounter++; - - if (recordCounter == 1) { - assertThat(result.f0).isEqualTo("this is"); - assertThat(result.f1).isOne(); - assertThat(result.f2).isEqualTo(new Double(2.0)); - assertThat((long) format.getCurrentState()).isEqualTo(15); - } else if (recordCounter == 2) { - assertThat(result.f0).isEqualTo("a test"); - assertThat(result.f1).isEqualTo(Integer.valueOf(3)); - assertThat(result.f2).isEqualTo(new Double(4.0)); - assertThat((long) format.getCurrentState()).isEqualTo(29); - } else if (recordCounter == 3) { - assertThat(result.f0).isEqualTo("#next"); - assertThat(result.f1).isEqualTo(Integer.valueOf(5)); - assertThat(result.f2).isEqualTo(new Double(6.0)); - assertThat((long) format.getCurrentState()).isEqualTo(42); - } else { - assertThat(result.f0).isEqualTo("asdadas"); - assertThat(result.f1).isEqualTo(new Integer(5)); - assertThat(result.f2).isEqualTo(new Double(30.0)); - assertThat((long) format.getCurrentState()).isEqualTo(58); - } - - // simulate checkpoint - Long state = format.getCurrentState(); - long offsetToRestore = state; - - // create a new format - format = - new TupleCsvInputFormat<>( - new Path(tempFile.toURI()), "\n", "|", typeInfo); - format.setLenient(true); - format.setBufferSize(bufferSize); - format.configure(config); - - // simulate the restore operation. - format.reopen(inputSplit, offsetToRestore); - } else { - result = new Tuple3<>(); - } - } - format.close(); - } - assertThat(recordCounter).isEqualTo(4); - } - - @Test - void ignoreInvalidLinesAndGetOffsetInLargeBuffer() { - ignoreInvalidLines(1024 * 1024); - } - - @Test - void ignoreInvalidLinesAndGetOffsetInSmallBuffer() { - ignoreInvalidLines(2); - } - - private void ignoreInvalidLines(int bufferSize) { - try { - final String fileContent = - "#description of the data\n" - + "header1|header2|header3|\n" - + "this is|1|2.0|\n" - + "//a comment\n" - + "a test|3|4.0|\n" - + "#next|5|6.0|\n" - + "asdasdas"; - - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class, Double.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, "\n", "|", typeInfo); - format.setLenient(true); - format.setBufferSize(bufferSize); - - final Configuration parameters = new Configuration(); - format.configure(parameters); - format.open(split); - - Tuple3 result = new Tuple3<>(); - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("this is"); - assertThat(result.f1).isOne(); - assertThat(result.f2).isEqualTo(new Double(2.0)); - assertThat((long) format.getCurrentState()).isEqualTo(65); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("a test"); - assertThat(result.f1).isEqualTo(Integer.valueOf(3)); - assertThat(result.f2).isEqualTo(new Double(4.0)); - assertThat((long) format.getCurrentState()).isEqualTo(91); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("#next"); - assertThat(result.f1).isEqualTo(Integer.valueOf(5)); - assertThat(result.f2).isEqualTo(new Double(6.0)); - assertThat((long) format.getCurrentState()).isEqualTo(104); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat((long) format.getCurrentState()).isEqualTo(fileContent.length()); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void ignoreSingleCharPrefixComments() { - try { - final String fileContent = - "#description of the data\n" - + "#successive commented line\n" - + "this is|1|2.0|\n" - + "a test|3|4.0|\n" - + "#next|5|6.0|\n"; - - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class, Double.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, "\n", "|", typeInfo); - format.setCommentPrefix("#"); - - final Configuration parameters = new Configuration(); - format.configure(parameters); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("this is"); - assertThat(result.f1).isOne(); - assertThat(result.f2).isEqualTo(new Double(2.0)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("a test"); - assertThat(result.f1).isEqualTo(Integer.valueOf(3)); - assertThat(result.f2).isEqualTo(new Double(4.0)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void ignoreMultiCharPrefixComments() { - try { - - final String fileContent = - "//description of the data\n" - + "//successive commented line\n" - + "this is|1|2.0|\n" - + "a test|3|4.0|\n" - + "//next|5|6.0|\n"; - - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, Integer.class, Double.class); - final CsvInputFormat> format = - new TupleCsvInputFormat>( - PATH, "\n", "|", typeInfo); - format.setCommentPrefix("//"); - - final Configuration parameters = new Configuration(); - format.configure(parameters); - format.open(split); - - Tuple3 result = new Tuple3(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("this is"); - assertThat(result.f1).isOne(); - assertThat(result.f2).isEqualTo(new Double(2.0)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("a test"); - assertThat(result.f1).isEqualTo(Integer.valueOf(3)); - assertThat(result.f2).isEqualTo(new Double(4.0)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void readStringFields() { - try { - final String fileContent = "abc|def|ghijk\nabc||hhg\n|||"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class, String.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, "\n", "|", typeInfo); - - final Configuration parameters = new Configuration(); - format.configure(parameters); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("abc"); - assertThat(result.f1).isEqualTo("def"); - assertThat(result.f2).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("abc"); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEqualTo("hhg"); - - result = format.nextRecord(result); - assertThat(result.f0).isEmpty(); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEmpty(); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void readMixedQuotedStringFields() { - try { - final String fileContent = "@a|b|c@|def|@ghijk@\nabc||@|hhg@\n|||"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class, String.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, "\n", "|", typeInfo); - - final Configuration parameters = new Configuration(); - format.enableQuotedStringParsing('@'); - format.configure(parameters); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.f0).isEqualTo("a|b|c"); - assertThat(result.f1).isEqualTo("def"); - assertThat(result.f2).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.f0).isEqualTo("abc"); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEqualTo("|hhg"); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.f0).isEmpty(); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEmpty(); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void readStringFieldsWithTrailingDelimiters() { - try { - final String fileContent = "abc|-def|-ghijk\nabc|-|-hhg\n|-|-|-\n"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class, String.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, typeInfo); - - format.setFieldDelimiter("|-"); - - format.configure(new Configuration()); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("abc"); - assertThat(result.f1).isEqualTo("def"); - assertThat(result.f2).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("abc"); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEqualTo("hhg"); - - result = format.nextRecord(result); - assertThat(result.f0).isEmpty(); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEmpty(); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testTailingEmptyFields() throws Exception { - final String fileContent = - "aa,bb,cc\n" - + // ok - "aa,bb,\n" - + // the last field is empty - "aa,,\n" - + // the last two fields are empty - ",,\n" - + // all fields are empty - "aa,bb"; // row too short - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class, String.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, typeInfo); - - format.setFieldDelimiter(","); - - format.configure(new Configuration()); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("aa"); - assertThat(result.f1).isEqualTo("bb"); - assertThat(result.f2).isEqualTo("cc"); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("aa"); - assertThat(result.f1).isEqualTo("bb"); - assertThat(result.f2).isEmpty(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo("aa"); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEmpty(); - - result = format.nextRecord(result); - assertThat(result.f0).isEmpty(); - assertThat(result.f1).isEmpty(); - assertThat(result.f2).isEmpty(); - - try { - format.nextRecord(result); - fail("Parse Exception was not thrown! (Row too short)"); - } catch (ParseException e) { - } - } - - @Test - void testIntegerFields() { - try { - final String fileContent = "111|222|333|444|555\n666|777|888|999|000|\n"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Integer.class, - Integer.class, - Integer.class, - Integer.class, - Integer.class); - final CsvInputFormat> format = - new TupleCsvInputFormat>( - PATH, typeInfo); - - format.setFieldDelimiter("|"); - - format.configure(new Configuration()); - format.open(split); - - Tuple5 result = new Tuple5<>(); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.f0).isEqualTo(Integer.valueOf(111)); - assertThat(result.f1).isEqualTo(Integer.valueOf(222)); - assertThat(result.f2).isEqualTo(Integer.valueOf(333)); - assertThat(result.f3).isEqualTo(Integer.valueOf(444)); - assertThat(result.f4).isEqualTo(Integer.valueOf(555)); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.f0).isEqualTo(Integer.valueOf(666)); - assertThat(result.f1).isEqualTo(Integer.valueOf(777)); - assertThat(result.f2).isEqualTo(Integer.valueOf(888)); - assertThat(result.f3).isEqualTo(Integer.valueOf(999)); - assertThat(result.f4).isEqualTo(Integer.valueOf(000)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testEmptyFields() { - try { - final String fileContent = - "|0|0|0|0|0|\n" - + "1||1|1|1|1|\n" - + "2|2||2|2|2|\n" - + "3|3|3| |3|3|\n" - + "4|4|4|4||4|\n" - + "5|5|5|5|5||\n"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Short.class, - Integer.class, - Long.class, - Float.class, - Double.class, - Byte.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, typeInfo); - - format.setFieldDelimiter("|"); - - format.configure(new Configuration()); - format.open(split); - - Tuple6 result = new Tuple6<>(); - - try { - result = format.nextRecord(result); - fail("Empty String Parse Exception was not thrown! (ShortParser)"); - } catch (ParseException e) { - } - try { - result = format.nextRecord(result); - fail("Empty String Parse Exception was not thrown! (IntegerParser)"); - } catch (ParseException e) { - } - try { - result = format.nextRecord(result); - fail("Empty String Parse Exception was not thrown! (LongParser)"); - } catch (ParseException e) { - } - try { - result = format.nextRecord(result); - fail("Empty String Parse Exception was not thrown! (FloatParser)"); - } catch (ParseException e) { - } - try { - result = format.nextRecord(result); - fail("Empty String Parse Exception was not thrown! (DoubleParser)"); - } catch (ParseException e) { - } - try { - result = format.nextRecord(result); - fail("Empty String Parse Exception was not thrown! (ByteParser)"); - } catch (ParseException e) { - } - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testDoubleFields() { - try { - final String fileContent = "11.1|22.2|33.3|44.4|55.5\n66.6|77.7|88.8|99.9|00.0|\n"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Double.class, Double.class, Double.class, Double.class, Double.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, typeInfo); - - format.setFieldDelimiter("|"); - - format.configure(new Configuration()); - format.open(split); - - Tuple5 result = new Tuple5<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Double.valueOf(11.1)); - assertThat(result.f1).isEqualTo(Double.valueOf(22.2)); - assertThat(result.f2).isEqualTo(Double.valueOf(33.3)); - assertThat(result.f3).isEqualTo(Double.valueOf(44.4)); - assertThat(result.f4).isEqualTo(Double.valueOf(55.5)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Double.valueOf(66.6)); - assertThat(result.f1).isEqualTo(Double.valueOf(77.7)); - assertThat(result.f2).isEqualTo(Double.valueOf(88.8)); - assertThat(result.f3).isEqualTo(Double.valueOf(99.9)); - assertThat(result.f4).isEqualTo(Double.valueOf(00.0)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testReadFirstN() { - try { - final String fileContent = "111|222|333|444|555|\n666|777|888|999|000|\n"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(Integer.class, Integer.class); - final CsvInputFormat> format = - new TupleCsvInputFormat>(PATH, typeInfo); - - format.setFieldDelimiter("|"); - - format.configure(new Configuration()); - format.open(split); - - Tuple2 result = new Tuple2<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(111)); - assertThat(result.f1).isEqualTo(Integer.valueOf(222)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(666)); - assertThat(result.f1).isEqualTo(Integer.valueOf(777)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testReadSparseWithNullFieldsForTypes() { - try { - final String fileContent = - "111|x|222|x|333|x|444|x|555|x|666|x|777|x|888|x|999|x|000|x|\n" - + "000|x|999|x|888|x|777|x|666|x|555|x|444|x|333|x|222|x|111|x|"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Integer.class, Integer.class, Integer.class); - final CsvInputFormat> format = - new TupleCsvInputFormat>( - PATH, - typeInfo, - new boolean[] {true, false, false, true, false, false, false, true}); - - format.setFieldDelimiter("|x|"); - - format.configure(new Configuration()); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(111)); - assertThat(result.f1).isEqualTo(Integer.valueOf(444)); - assertThat(result.f2).isEqualTo(Integer.valueOf(888)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(000)); - assertThat(result.f1).isEqualTo(Integer.valueOf(777)); - assertThat(result.f2).isEqualTo(Integer.valueOf(333)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testReadSparseWithPositionSetter() { - try { - final String fileContent = - "111|222|333|444|555|666|777|888|999|000|\n000|999|888|777|666|555|444|333|222|111|"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Integer.class, Integer.class, Integer.class); - final CsvInputFormat> format = - new TupleCsvInputFormat>( - PATH, typeInfo, new int[] {0, 3, 7}); - - format.setFieldDelimiter("|"); - - format.configure(new Configuration()); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(111)); - assertThat(result.f1).isEqualTo(Integer.valueOf(444)); - assertThat(result.f2).isEqualTo(Integer.valueOf(888)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(000)); - assertThat(result.f1).isEqualTo(Integer.valueOf(777)); - assertThat(result.f2).isEqualTo(Integer.valueOf(333)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testReadSparseWithMask() { - try { - final String fileContent = - "111&&222&&333&&444&&555&&666&&777&&888&&999&&000&&\n" - + "000&&999&&888&&777&&666&&555&&444&&333&&222&&111&&"; - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Integer.class, Integer.class, Integer.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>( - PATH, - typeInfo, - new boolean[] {true, false, false, true, false, false, false, true}); - - format.setFieldDelimiter("&&"); - - format.configure(new Configuration()); - format.open(split); - - Tuple3 result = new Tuple3<>(); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(111)); - assertThat(result.f1).isEqualTo(Integer.valueOf(444)); - assertThat(result.f2).isEqualTo(Integer.valueOf(888)); - - result = format.nextRecord(result); - assertThat(result.f0).isEqualTo(Integer.valueOf(000)); - assertThat(result.f1).isEqualTo(Integer.valueOf(777)); - assertThat(result.f2).isEqualTo(Integer.valueOf(333)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testParseStringErrors() { - StringParser stringParser = new StringParser(); - stringParser.enableQuotedStringParsing((byte) '"'); - - Object[][] failures = { - {"\"string\" trailing", FieldParser.ParseErrorState.UNQUOTED_CHARS_AFTER_QUOTED_STRING}, - {"\"unterminated ", FieldParser.ParseErrorState.UNTERMINATED_QUOTED_STRING} - }; - - for (Object[] failure : failures) { - String input = (String) failure[0]; - - int result = - stringParser.parseField( - input.getBytes(ConfigConstants.DEFAULT_CHARSET), - 0, - input.length(), - new byte[] {'|'}, - null); - - assertThat(result).isEqualTo(-1); - assertThat(stringParser.getErrorState()).isSameAs(failure[1]); - } - } - - @Disabled("Test disabled because we do not support double-quote escaped quotes right now.") - @Test - void testParserCorrectness() throws Exception { - // RFC 4180 Compliance Test content - // Taken from http://en.wikipedia.org/wiki/Comma-separated_values#Example - final String fileContent = - "Year,Make,Model,Description,Price\n" - + "1997,Ford,E350,\"ac, abs, moon\",3000.00\n" - + "1999,Chevy,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00\n" - + "1996,Jeep,Grand Cherokee,\"MUST SELL! air, moon roof, loaded\",4799.00\n" - + "1999,Chevy,\"Venture \"\"Extended Edition, Very Large\"\"\",,5000.00\n" - + ",,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00"; - - final FileInputSplit split = createTempFile(fileContent); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo( - Integer.class, String.class, String.class, String.class, Double.class); - final CsvInputFormat> format = - new TupleCsvInputFormat<>(PATH, typeInfo); - - format.setSkipFirstLineAsHeader(true); - format.setFieldDelimiter(","); - - format.configure(new Configuration()); - format.open(split); - - Tuple5 result = new Tuple5<>(); - - @SuppressWarnings("unchecked") - Tuple5[] expectedLines = - new Tuple5[] { - new Tuple5<>(1997, "Ford", "E350", "ac, abs, moon", 3000.0), - new Tuple5<>(1999, "Chevy", "Venture \"Extended Edition\"", "", 4900.0), - new Tuple5<>( - 1996, - "Jeep", - "Grand Cherokee", - "MUST SELL! air, moon roof, loaded", - 4799.00), - new Tuple5<>( - 1999, "Chevy", "Venture \"Extended Edition, Very Large\"", "", 5000.00), - new Tuple5<>(0, "", "Venture \"Extended Edition\"", "", 4900.0) - }; - - try { - for (Tuple5 expected : expectedLines) { - result = format.nextRecord(result); - assertThat(result).isEqualTo(expected); - } - - assertThat(format.nextRecord(result)).isNull(); - assertThat(format.reachedEnd()).isTrue(); - - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - private FileInputSplit createTempFile(String content) throws IOException { - File tempFile = File.createTempFile("test_contents", "tmp"); - tempFile.deleteOnExit(); - - OutputStreamWriter wrt = - new OutputStreamWriter(new FileOutputStream(tempFile), StandardCharsets.UTF_8); - wrt.write(content); - wrt.close(); - - return new FileInputSplit( - 0, - new Path(tempFile.toURI().toString()), - 0, - tempFile.length(), - new String[] {"localhost"}); - } - - @Test - void testWindowsLineEndRemoval() { - - // Check typical use case -- linux file is correct and it is set up to linux (\n) - this.testRemovingTrailingCR("\n", "\n"); - - // Check typical windows case -- windows file endings and file has windows file endings set - // up - this.testRemovingTrailingCR("\r\n", "\r\n"); - - // Check problematic case windows file -- windows file endings (\r\n) but linux line endings - // (\n) set up - this.testRemovingTrailingCR("\r\n", "\n"); - - // Check problematic case linux file -- linux file endings (\n) but windows file endings set - // up (\r\n) - // Specific setup for windows line endings will expect \r\n because it has to be set up and - // is not standard. - } - - private void testRemovingTrailingCR(String lineBreakerInFile, String lineBreakerSetup) { - File tempFile = null; - - String fileContent = - CsvInputFormatTest.FIRST_PART - + lineBreakerInFile - + CsvInputFormatTest.SECOND_PART - + lineBreakerInFile; - - try { - // create input file - tempFile = File.createTempFile("CsvInputFormatTest", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write(fileContent); - wrt.close(); - - final TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class); - final CsvInputFormat> inputFormat = - new TupleCsvInputFormat<>(new Path(tempFile.toURI().toString()), typeInfo); - - Configuration parameters = new Configuration(); - inputFormat.configure(parameters); - - inputFormat.setDelimiter(lineBreakerSetup); - - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - Tuple1 result = inputFormat.nextRecord(new Tuple1<>()); - - assertThat(result).as("Expecting to not return null").isNotNull(); - - assertThat(result.f0).isEqualTo(FIRST_PART); - - result = inputFormat.nextRecord(result); - - assertThat(result).as("Expecting to not return null").isNotNull(); - assertThat(result.f0).isEqualTo(SECOND_PART); - - } catch (Throwable t) { - System.err.println("test failed with exception: " + t.getMessage()); - t.printStackTrace(System.err); - fail("Test erroneous"); - } - } - - private void validatePojoItem(CsvInputFormat format) throws Exception { - PojoItem item = new PojoItem(); - - format.nextRecord(item); - - assertThat(item.field1).isEqualTo(123); - assertThat(item.field2).isEqualTo("AAA"); - assertThat(item.field3).isEqualTo(Double.valueOf(3.123)); - assertThat(item.field4).isEqualTo("BBB"); - - format.nextRecord(item); - - assertThat(item.field1).isEqualTo(456); - assertThat(item.field2).isEqualTo("BBB"); - assertThat(item.field3).isEqualTo(Double.valueOf(1.123)); - assertThat(item.field4).isEqualTo("AAA"); - } - - @Test - void testPojoType() throws Exception { - File tempFile = File.createTempFile("CsvReaderPojoType", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write("123,AAA,3.123,BBB\n"); - wrt.write("456,BBB,1.123,AAA\n"); - wrt.close(); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PojoItem.class); - CsvInputFormat inputFormat = - new PojoCsvInputFormat(new Path(tempFile.toURI().toString()), typeInfo); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - validatePojoItem(inputFormat); - } - - @Test - void testPojoTypeWithPrivateField() throws Exception { - File tempFile = File.createTempFile("CsvReaderPojoType", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write("123,AAA,3.123,BBB\n"); - wrt.write("456,BBB,1.123,AAA\n"); - wrt.close(); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PrivatePojoItem.class); - CsvInputFormat inputFormat = - new PojoCsvInputFormat<>(new Path(tempFile.toURI().toString()), typeInfo); - - inputFormat.configure(new Configuration()); - - FileInputSplit[] splits = inputFormat.createInputSplits(1); - inputFormat.open(splits[0]); - - PrivatePojoItem item = new PrivatePojoItem(); - inputFormat.nextRecord(item); - - assertThat(item.field1).isEqualTo(123); - assertThat(item.field2).isEqualTo("AAA"); - assertThat(item.field3).isEqualTo(Double.valueOf(3.123)); - assertThat(item.field4).isEqualTo("BBB"); - - inputFormat.nextRecord(item); - - assertThat(item.field1).isEqualTo(456); - assertThat(item.field2).isEqualTo("BBB"); - assertThat(item.field3).isEqualTo(Double.valueOf(1.123)); - assertThat(item.field4).isEqualTo("AAA"); - } - - @Test - void testPojoTypeWithTrailingEmptyFields() throws Exception { - final String fileContent = "123,,3.123,,\n456,BBB,3.23,,"; - final FileInputSplit split = createTempFile(fileContent); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PrivatePojoItem.class); - CsvInputFormat inputFormat = new PojoCsvInputFormat<>(PATH, typeInfo); - - inputFormat.configure(new Configuration()); - inputFormat.open(split); - - PrivatePojoItem item = new PrivatePojoItem(); - inputFormat.nextRecord(item); - - assertThat(item.field1).isEqualTo(123); - assertThat(item.field2).isEmpty(); - assertThat(item.field3).isEqualTo(Double.valueOf(3.123)); - assertThat(item.field4).isEmpty(); - - inputFormat.nextRecord(item); - - assertThat(item.field1).isEqualTo(456); - assertThat(item.field2).isEqualTo("BBB"); - assertThat(item.field3).isEqualTo(Double.valueOf(3.23)); - assertThat(item.field4).isEmpty(); - } - - @Test - void testPojoTypeWithMappingInformation() throws Exception { - File tempFile = File.createTempFile("CsvReaderPojoType", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write("123,3.123,AAA,BBB\n"); - wrt.write("456,1.123,BBB,AAA\n"); - wrt.close(); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PojoItem.class); - CsvInputFormat inputFormat = - new PojoCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new String[] {"field1", "field3", "field2", "field4"}); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - validatePojoItem(inputFormat); - } - - @Test - void testPojoTypeWithPartialFieldInCSV() throws Exception { - File tempFile = File.createTempFile("CsvReaderPojoType", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write("123,NODATA,AAA,NODATA,3.123,BBB\n"); - wrt.write("456,NODATA,BBB,NODATA,1.123,AAA\n"); - wrt.close(); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PojoItem.class); - CsvInputFormat inputFormat = - new PojoCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new boolean[] {true, false, true, false, true, true}); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - validatePojoItem(inputFormat); - } - - @Test - void testPojoTypeWithMappingInfoAndPartialField() throws Exception { - File tempFile = File.createTempFile("CsvReaderPojoType", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write("123,3.123,AAA,BBB\n"); - wrt.write("456,1.123,BBB,AAA\n"); - wrt.close(); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PojoItem.class); - CsvInputFormat inputFormat = - new PojoCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new String[] {"field1", "field4"}, - new boolean[] {true, false, false, true}); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - PojoItem item = new PojoItem(); - inputFormat.nextRecord(item); - - assertThat(item.field1).isEqualTo(123); - assertThat(item.field4).isEqualTo("BBB"); - } - - @Test - void testPojoTypeWithInvalidFieldMapping() throws Exception { - File tempFile = File.createTempFile("CsvReaderPojoType", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(PojoItem.class); - - try { - new PojoCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new String[] {"field1", "field2"}); - fail("The number of POJO fields cannot be same as that of selected CSV fields"); - } catch (IllegalArgumentException e) { - // success - } - - try { - new PojoCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new String[] {"field1", "field2", null, "field4"}); - fail("Fields mapping cannot contain null."); - } catch (NullPointerException e) { - // success - } - - try { - new PojoCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new String[] {"field1", "field2", "field3", "field5"}); - fail("Invalid field name"); - } catch (IllegalArgumentException e) { - // success - } - } - - @Test - void testQuotedStringParsingWithIncludeFields() throws Exception { - final String fileContent = - "\"20:41:52-1-3-2015\"|\"Re: Taskmanager memory error in Eclipse\"|" - + "\"Blahblah \"|\"blaaa|\"blubb\""; - - final File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); - writer.write(fileContent); - writer.close(); - - TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class); - CsvInputFormat> inputFormat = - new TupleCsvInputFormat<>( - new Path(tempFile.toURI().toString()), - typeInfo, - new boolean[] {true, false, true}); - - inputFormat.enableQuotedStringParsing('"'); - inputFormat.setFieldDelimiter("|"); - inputFormat.setDelimiter('\n'); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - Tuple2 record = inputFormat.nextRecord(new Tuple2<>()); - - assertThat(record.f0).isEqualTo("20:41:52-1-3-2015"); - assertThat(record.f1).isEqualTo("Blahblah "); - } - - @Test - void testQuotedStringParsingWithEscapedQuotes() throws Exception { - final String fileContent = "\"\\\"Hello\\\" World\"|\"We are\\\" young\""; - - final File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); - writer.write(fileContent); - writer.close(); - - TupleTypeInfo> typeInfo = - TupleTypeInfo.getBasicTupleTypeInfo(String.class, String.class); - CsvInputFormat> inputFormat = - new TupleCsvInputFormat<>(new Path(tempFile.toURI().toString()), typeInfo); - - inputFormat.enableQuotedStringParsing('"'); - inputFormat.setFieldDelimiter("|"); - inputFormat.setDelimiter('\n'); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - Tuple2 record = inputFormat.nextRecord(new Tuple2<>()); - - assertThat(record.f0).isEqualTo("\\\"Hello\\\" World"); - assertThat(record.f1).isEqualTo("We are\\\" young"); - } - - /** - * Tests that the CSV input format can deal with POJOs which are subclasses. - * - * @throws Exception - */ - @Test - void testPojoSubclassType() throws Exception { - final String fileContent = "t1,foobar,tweet2\nt2,barfoo,tweet2"; - - final File tempFile = File.createTempFile("CsvReaderPOJOSubclass", "tmp"); - tempFile.deleteOnExit(); - - OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); - writer.write(fileContent); - writer.close(); - - @SuppressWarnings("unchecked") - PojoTypeInfo typeInfo = - (PojoTypeInfo) TypeExtractor.createTypeInfo(TwitterPOJO.class); - CsvInputFormat inputFormat = - new PojoCsvInputFormat<>(new Path(tempFile.toURI().toString()), typeInfo); - - inputFormat.configure(new Configuration()); - FileInputSplit[] splits = inputFormat.createInputSplits(1); - - inputFormat.open(splits[0]); - - List expected = new ArrayList<>(); - - for (String line : fileContent.split("\n")) { - String[] elements = line.split(","); - expected.add(new TwitterPOJO(elements[0], elements[1], elements[2])); - } - - List actual = new ArrayList<>(); - - TwitterPOJO pojo; - - while ((pojo = inputFormat.nextRecord(new TwitterPOJO())) != null) { - actual.add(pojo); - } - - assertThat(actual).isEqualTo(expected); - } - - // -------------------------------------------------------------------------------------------- - // Custom types for testing - // -------------------------------------------------------------------------------------------- - - /** Sample test pojo. */ - public static class PojoItem { - public int field1; - public String field2; - public Double field3; - public String field4; - } - - /** Sample test pojo with private fields. */ - public static class PrivatePojoItem { - private int field1; - private String field2; - private Double field3; - private String field4; - - public int getField1() { - return field1; - } - - public void setField1(int field1) { - this.field1 = field1; - } - - public String getField2() { - return field2; - } - - public void setField2(String field2) { - this.field2 = field2; - } - - public Double getField3() { - return field3; - } - - public void setField3(Double field3) { - this.field3 = field3; - } - - public String getField4() { - return field4; - } - - public void setField4(String field4) { - this.field4 = field4; - } - } - - /** Sample test pojo. */ - public static class POJO { - public String table; - public String time; - - public POJO() { - this("", ""); - } - - public POJO(String table, String time) { - this.table = table; - this.time = time; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof POJO) { - POJO other = (POJO) obj; - return table.equals(other.table) && time.equals(other.time); - } else { - return false; - } - } - } - - /** Sample test pojo representing tweets. */ - public static class TwitterPOJO extends POJO { - public String tweet; - - public TwitterPOJO() { - this("", "", ""); - } - - public TwitterPOJO(String table, String time, String tweet) { - super(table, time); - this.tweet = tweet; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof TwitterPOJO) { - TwitterPOJO other = (TwitterPOJO) obj; - - return super.equals(other) && tweet.equals(other.tweet); - } else { - return false; - } - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CsvOutputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CsvOutputFormatTest.java deleted file mode 100644 index 9268ca400d30b..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/CsvOutputFormatTest.java +++ /dev/null @@ -1,96 +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.api.java.io; - -import org.apache.flink.api.common.io.FileOutputFormat; -import org.apache.flink.api.common.io.FirstAttemptInitializationContext; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link CsvOutputFormat}. */ -class CsvOutputFormatTest { - - private String path = null; - - @BeforeEach - void createFile() throws Exception { - path = File.createTempFile("csv_output_test_file", ".csv").getAbsolutePath(); - } - - @Test - void testNullAllow() throws Exception { - final CsvOutputFormat> csvOutputFormat = - new CsvOutputFormat<>(new Path(path)); - try { - csvOutputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE); - csvOutputFormat.setOutputDirectoryMode(FileOutputFormat.OutputDirectoryMode.PARONLY); - csvOutputFormat.setAllowNullValues(true); - csvOutputFormat.open(FirstAttemptInitializationContext.of(0, 1)); - csvOutputFormat.writeRecord(new Tuple3<>("One", null, 8)); - } finally { - csvOutputFormat.close(); - } - - java.nio.file.Path p = Paths.get(path); - assertThat(Files.exists(p)).isTrue(); - List lines = Files.readAllLines(Paths.get(path), StandardCharsets.UTF_8); - assertThat(lines).hasSize(1); - assertThat(lines.get(0)).isEqualTo("One,,8"); - } - - @Test - void testNullDisallowOnDefault() throws Exception { - final CsvOutputFormat> csvOutputFormat = - new CsvOutputFormat<>(new Path(path)); - try { - csvOutputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE); - csvOutputFormat.setOutputDirectoryMode(FileOutputFormat.OutputDirectoryMode.PARONLY); - csvOutputFormat.open(FirstAttemptInitializationContext.of(0, 1)); - try { - csvOutputFormat.writeRecord(new Tuple3<>("One", null, 8)); - fail("should fail with an exception"); - } catch (RuntimeException e) { - // expected - } - - } finally { - csvOutputFormat.close(); - } - } - - @AfterEach - void cleanUp() throws IOException { - Files.deleteIfExists(Paths.get(path)); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java deleted file mode 100644 index 6c086da73d69a..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java +++ /dev/null @@ -1,63 +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.api.java.io; - -import org.apache.flink.api.java.ExecutionEnvironment; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link ExecutionEnvironment#fromElements}. */ -class FromElementsTest { - - @Test - void fromElementsWithBaseTypeTest1() { - ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment(); - executionEnvironment.fromElements( - ParentType.class, new SubType(1, "Java"), new ParentType(1, "hello")); - } - - @Test - void fromElementsWithBaseTypeTest2() { - ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment(); - assertThatThrownBy( - () -> - executionEnvironment.fromElements( - SubType.class, - new SubType(1, "Java"), - new ParentType(1, "hello"))) - .isInstanceOf(IllegalArgumentException.class); - } - - private static class ParentType { - int num; - String string; - - public ParentType(int num, String string) { - this.num = num; - this.string = string; - } - } - - private static class SubType extends ParentType { - public SubType(int num, String string) { - super(num, string); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/PrimitiveInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/PrimitiveInputFormatTest.java deleted file mode 100644 index 68e502e5be47d..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/PrimitiveInputFormatTest.java +++ /dev/null @@ -1,192 +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.api.java.io; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; - -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link PrimitiveInputFormat}. */ -class PrimitiveInputFormatTest { - - private static final Path PATH = new Path("an/ignored/file/"); - - @Test - void testStringInput() { - try { - final String fileContent = "abc||def||||"; - final FileInputSplit split = createInputSplit(fileContent); - - final PrimitiveInputFormat format = - new PrimitiveInputFormat<>(PATH, "||", String.class); - - final Configuration parameters = new Configuration(); - format.configure(parameters); - format.open(split); - - String result = null; - - result = format.nextRecord(result); - assertThat(result).isEqualTo("abc"); - - result = format.nextRecord(result); - assertThat(result).isEqualTo("def"); - - result = format.nextRecord(result); - assertThat(result).isEmpty(); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - ex.printStackTrace(); - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testIntegerInput() { - try { - final String fileContent = "111|222|"; - final FileInputSplit split = createInputSplit(fileContent); - - final PrimitiveInputFormat format = - new PrimitiveInputFormat<>(PATH, "|", Integer.class); - - format.configure(new Configuration()); - format.open(split); - - Integer result = null; - result = format.nextRecord(result); - assertThat(result).isEqualTo(Integer.valueOf(111)); - - result = format.nextRecord(result); - assertThat(result).isEqualTo(Integer.valueOf(222)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testDoubleInputLinewise() { - try { - final String fileContent = "1.21\n2.23\n"; - final FileInputSplit split = createInputSplit(fileContent); - - final PrimitiveInputFormat format = - new PrimitiveInputFormat<>(PATH, Double.class); - - format.configure(new Configuration()); - format.open(split); - - Double result = null; - result = format.nextRecord(result); - assertThat(result).isEqualTo(Double.valueOf(1.21)); - - result = format.nextRecord(result); - assertThat(result).isEqualTo(Double.valueOf(2.23)); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testRemovingTrailingCR() { - try { - String first = "First line"; - String second = "Second line"; - String fileContent = first + "\r\n" + second + "\r\n"; - final FileInputSplit split = createInputSplit(fileContent); - - final PrimitiveInputFormat format = - new PrimitiveInputFormat<>(PATH, String.class); - - format.configure(new Configuration()); - format.open(split); - - String result = null; - - result = format.nextRecord(result); - assertThat(result).isEqualTo(first); - - result = format.nextRecord(result); - assertThat(result).isEqualTo(second); - } catch (Exception ex) { - fail("Test failed due to a " + ex.getClass().getName() + ": " + ex.getMessage()); - } - } - - @Test - void testFailingInput() throws IOException { - - final String fileContent = "111|222|asdf|17"; - final FileInputSplit split = createInputSplit(fileContent); - - final PrimitiveInputFormat format = - new PrimitiveInputFormat<>(PATH, "|", Integer.class); - - format.configure(new Configuration()); - format.open(split); - - Integer result = null; - result = format.nextRecord(result); - assertThat(result).isEqualTo(Integer.valueOf(111)); - - result = format.nextRecord(result); - assertThat(result).isEqualTo(Integer.valueOf(222)); - - final Integer finalResult = result; - - assertThatThrownBy(() -> format.nextRecord(finalResult)).isInstanceOf(IOException.class); - } - - private FileInputSplit createInputSplit(String content) throws IOException { - File tempFile = File.createTempFile("test_contents", "tmp"); - tempFile.deleteOnExit(); - - try (FileWriter wrt = new FileWriter(tempFile)) { - wrt.write(content); - } - - return new FileInputSplit( - 0, - new Path(tempFile.toURI().toString()), - 0, - tempFile.length(), - new String[] {"localhost"}); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/PrintingOutputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/PrintingOutputFormatTest.java deleted file mode 100644 index 6ce7ff6638e1d..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/PrintingOutputFormatTest.java +++ /dev/null @@ -1,119 +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.api.java.io; - -import org.apache.flink.api.common.io.FirstAttemptInitializationContext; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for the {@link PrintingOutputFormat}. */ -class PrintingOutputFormatTest { - - private final PrintStream originalSystemOut = System.out; - private final PrintStream originalSystemErr = System.err; - - private final ByteArrayOutputStream arrayOutputStream = new ByteArrayOutputStream(); - private final ByteArrayOutputStream arrayErrorStream = new ByteArrayOutputStream(); - - private final String line = System.lineSeparator(); - - @BeforeEach - void setUp() { - System.setOut(new PrintStream(arrayOutputStream)); - System.setErr(new PrintStream(arrayErrorStream)); - } - - @AfterEach - void tearDown() { - if (System.out != originalSystemOut) { - System.out.close(); - } - if (System.err != originalSystemErr) { - System.err.close(); - } - System.setOut(originalSystemOut); - System.setErr(originalSystemErr); - } - - @Test - void testPrintOutputFormatStdOut() throws Exception { - PrintingOutputFormat printSink = new PrintingOutputFormat<>(); - printSink.open(FirstAttemptInitializationContext.of(0, 1)); - - printSink.writeRecord("hello world!"); - - assertThat(printSink).hasToString("Print to System.out"); - assertThat(arrayOutputStream).hasToString("hello world!" + line); - printSink.close(); - } - - @Test - void testPrintOutputFormatStdErr() throws Exception { - PrintingOutputFormat printSink = new PrintingOutputFormat<>(true); - printSink.open(FirstAttemptInitializationContext.of(0, 1)); - - printSink.writeRecord("hello world!"); - - assertThat(printSink).hasToString("Print to System.err"); - assertThat(arrayErrorStream).hasToString("hello world!" + line); - printSink.close(); - } - - @Test - void testPrintOutputFormatWithPrefix() throws Exception { - PrintingOutputFormat printSink = new PrintingOutputFormat<>(); - printSink.open(FirstAttemptInitializationContext.of(1, 2)); - - printSink.writeRecord("hello world!"); - - assertThat(printSink).hasToString("Print to System.out"); - assertThat(arrayOutputStream).hasToString("2> hello world!" + line); - printSink.close(); - } - - @Test - void testPrintOutputFormatWithIdentifierAndPrefix() throws Exception { - PrintingOutputFormat printSink = new PrintingOutputFormat<>("mySink", false); - printSink.open(FirstAttemptInitializationContext.of(1, 2)); - - printSink.writeRecord("hello world!"); - - assertThat(printSink).hasToString("Print to System.out"); - assertThat(arrayOutputStream).hasToString("mySink:2> hello world!" + line); - printSink.close(); - } - - @Test - void testPrintOutputFormatWithIdentifierButNoPrefix() throws Exception { - PrintingOutputFormat printSink = new PrintingOutputFormat<>("mySink", false); - printSink.open(FirstAttemptInitializationContext.of(0, 1)); - - printSink.writeRecord("hello world!"); - - assertThat(printSink).hasToString("Print to System.out"); - assertThat(arrayOutputStream).hasToString("mySink> hello world!" + line); - printSink.close(); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java deleted file mode 100644 index d2b847d6f408c..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java +++ /dev/null @@ -1,1023 +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.api.java.io; - -import org.apache.flink.api.common.io.ParseException; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.types.Row; -import org.apache.flink.types.parser.FieldParser; -import org.apache.flink.types.parser.StringParser; - -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.nio.charset.StandardCharsets; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.HashMap; -import java.util.Map; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link RowCsvInputFormat}. */ -class RowCsvInputFormatTest { - - private static final Path PATH = new Path("an/ignored/file/"); - - // static variables for testing the removal of \r\n to \n - private static final String FIRST_PART = "That is the first part"; - private static final String SECOND_PART = "That is the second part"; - - @Test - void ignoreInvalidLines() throws Exception { - String fileContent = - "#description of the data\n" - + "header1|header2|header3|\n" - + "this is|1|2.0|\n" - + "//a comment\n" - + "a test|3|4.0|\n" - + "#next|5|6.0|\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.setLenient(false); - Configuration parameters = new Configuration(); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - try { - result = format.nextRecord(result); - fail("Parse Exception was not thrown! (Row too short)"); - } catch (ParseException ignored) { - } // => ok - - try { - result = format.nextRecord(result); - fail("Parse Exception was not thrown! (Invalid int value)"); - } catch (ParseException ignored) { - } // => ok - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("this is"); - assertThat(result.getField(1)).isEqualTo(1); - assertThat(result.getField(2)).isEqualTo(2.0); - - try { - result = format.nextRecord(result); - fail("Parse Exception was not thrown! (Row too short)"); - } catch (ParseException ignored) { - } // => ok - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("a test"); - assertThat(result.getField(1)).isEqualTo(3); - assertThat(result.getField(2)).isEqualTo(4.0); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("#next"); - assertThat(result.getField(1)).isEqualTo(5); - assertThat(result.getField(2)).isEqualTo(6.0); - - result = format.nextRecord(result); - assertThat(result).isNull(); - - // re-open with lenient = true - format.setLenient(true); - format.configure(parameters); - format.open(split); - - result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("header1"); - assertThat(result.getField(1)).isNull(); - assertThat(result.getField(2)).isNull(); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("this is"); - assertThat(result.getField(1)).isEqualTo(1); - assertThat(result.getField(2)).isEqualTo(2.0); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("a test"); - assertThat(result.getField(1)).isEqualTo(3); - assertThat(result.getField(2)).isEqualTo(4.0); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("#next"); - assertThat(result.getField(1)).isEqualTo(5); - assertThat(result.getField(2)).isEqualTo(6.0); - result = format.nextRecord(result); - assertThat(result).isNull(); - } - - @Test - void ignoreSingleCharPrefixComments() throws Exception { - String fileContent = - "#description of the data\n" - + "#successive commented line\n" - + "this is|1|2.0|\n" - + "a test|3|4.0|\n" - + "#next|5|6.0|\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.setCommentPrefix("#"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("this is"); - assertThat(result.getField(1)).isEqualTo(1); - assertThat(result.getField(2)).isEqualTo(2.0); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("a test"); - assertThat(result.getField(1)).isEqualTo(3); - assertThat(result.getField(2)).isEqualTo(4.0); - - result = format.nextRecord(result); - assertThat(result).isNull(); - } - - @Test - void ignoreMultiCharPrefixComments() throws Exception { - String fileContent = - "//description of the data\n" - + "//successive commented line\n" - + "this is|1|2.0|\n" - + "a test|3|4.0|\n" - + "//next|5|6.0|\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.setCommentPrefix("//"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("this is"); - assertThat(result.getField(1)).isEqualTo(1); - assertThat(result.getField(2)).isEqualTo(2.0); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("a test"); - assertThat(result.getField(1)).isEqualTo(3); - assertThat(result.getField(2)).isEqualTo(4.0); - result = format.nextRecord(result); - assertThat(result).isNull(); - } - - @Test - void readStringFields() throws Exception { - String fileContent = "abc|def|ghijk\nabc||hhg\n|||\n||"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo("def"); - assertThat(result.getField(2)).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo("hhg"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(""); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(""); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void readMixedQuotedStringFields() throws Exception { - String fileContent = "@a|b|c@|def|@ghijk@\nabc||@|hhg@\n|||\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.enableQuotedStringParsing('@'); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("a|b|c"); - assertThat(result.getField(1)).isEqualTo("def"); - assertThat(result.getField(2)).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo("|hhg"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(""); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void readStringFieldsWithTrailingDelimiters() throws Exception { - String fileContent = "abc|-def|-ghijk\nabc|-|-hhg\n|-|-|-\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.setFieldDelimiter("|-"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo("def"); - assertThat(result.getField(2)).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo("hhg"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(""); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testTailingEmptyFields() throws Exception { - String fileContent = - "abc|-def|-ghijk\n" - + "abc|-def|-\n" - + "abc|-|-\n" - + "|-|-|-\n" - + "|-|-\n" - + "abc|-def\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - format.setFieldDelimiter("|-"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo("def"); - assertThat(result.getField(2)).isEqualTo("ghijk"); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo("def"); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo("abc"); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(""); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(""); - assertThat(result.getField(1)).isEqualTo(""); - assertThat(result.getField(2)).isEqualTo(""); - - try { - format.nextRecord(result); - fail("Parse Exception was not thrown! (Row too short)"); - } catch (ParseException e) { - } - } - - @Test - void testIntegerFields() throws Exception { - String fileContent = "111|222|333|444|555\n666|777|888|999|000|\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|"); - - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(5); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(111); - assertThat(result.getField(1)).isEqualTo(222); - assertThat(result.getField(2)).isEqualTo(333); - assertThat(result.getField(3)).isEqualTo(444); - assertThat(result.getField(4)).isEqualTo(555); - - result = format.nextRecord(result); - assertThat(result.getField(0)).isEqualTo(666); - assertThat(result.getField(1)).isEqualTo(777); - assertThat(result.getField(2)).isEqualTo(888); - assertThat(result.getField(3)).isEqualTo(999); - assertThat(result.getField(4)).isEqualTo(0); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testEmptyFields() throws Exception { - String fileContent = - ",,,,,,,,\n" - + ",,,,,,,\n" - + ",,,,,,,,\n" - + ",,,,,,,\n" - + ",,,,,,,,\n" - + ",,,,,,,,\n" - + ",,,,,,,\n" - + ",,,,,,,,\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.BOOLEAN_TYPE_INFO, - BasicTypeInfo.BYTE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.FLOAT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.SHORT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, true); - format.setFieldDelimiter(","); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(8); - int linesCnt = fileContent.split("\n").length; - - for (int i = 0; i < linesCnt; i++) { - result = format.nextRecord(result); - assertThat(result.getField(i)).isNull(); - } - - // ensure no more rows - assertThat(format.nextRecord(result)).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testDoubleFields() throws Exception { - String fileContent = "11.1|22.2|33.3|44.4|55.5\n66.6|77.7|88.8|99.9|00.0|\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes); - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(5); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(11.1); - assertThat(result.getField(1)).isEqualTo(22.2); - assertThat(result.getField(2)).isEqualTo(33.3); - assertThat(result.getField(3)).isEqualTo(44.4); - assertThat(result.getField(4)).isEqualTo(55.5); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(66.6); - assertThat(result.getField(1)).isEqualTo(77.7); - assertThat(result.getField(2)).isEqualTo(88.8); - assertThat(result.getField(3)).isEqualTo(99.9); - assertThat(result.getField(4)).isEqualTo(0.0); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testReadFirstN() throws Exception { - String fileContent = "111|222|333|444|555|\n666|777|888|999|000|\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] {BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO}; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes); - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(2); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(111); - assertThat(result.getField(1)).isEqualTo(222); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(666); - assertThat(result.getField(1)).isEqualTo(777); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testReadSparseWithNullFieldsForTypes() throws Exception { - String fileContent = - "111|x|222|x|333|x|444|x|555|x|666|x|777|x|888|x|999|x|000|x|\n" - + "000|x|999|x|888|x|777|x|666|x|555|x|444|x|333|x|222|x|111|x|"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, new int[] {0, 3, 7}); - format.setFieldDelimiter("|x|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(111); - assertThat(result.getField(1)).isEqualTo(444); - assertThat(result.getField(2)).isEqualTo(888); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(0); - assertThat(result.getField(1)).isEqualTo(777); - assertThat(result.getField(2)).isEqualTo(333); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testReadSparseWithPositionSetter() throws Exception { - String fileContent = - "111|222|333|444|555|666|777|888|999|000|\n" - + "000|999|888|777|666|555|444|333|222|111|"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, new int[] {0, 3, 7}); - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - result = format.nextRecord(result); - - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(111); - assertThat(result.getField(1)).isEqualTo(444); - assertThat(result.getField(2)).isEqualTo(888); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(0); - assertThat(result.getField(1)).isEqualTo(777); - assertThat(result.getField(2)).isEqualTo(333); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testReadSparseWithMask() throws Exception { - String fileContent = - "111&&222&&333&&444&&555&&666&&777&&888&&999&&000&&\n" - + "000&&999&&888&&777&&666&&555&&444&&333&&222&&111&&"; - - FileInputSplit split = RowCsvInputFormatTest.createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, new int[] {0, 3, 7}); - format.setFieldDelimiter("&&"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(111); - assertThat(result.getField(1)).isEqualTo(444); - assertThat(result.getField(2)).isEqualTo(888); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(0); - assertThat(result.getField(1)).isEqualTo(777); - assertThat(result.getField(2)).isEqualTo(333); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testParseStringErrors() { - StringParser stringParser = new StringParser(); - - stringParser.enableQuotedStringParsing((byte) '"'); - - Map failures = new HashMap<>(); - failures.put( - "\"string\" trailing", - FieldParser.ParseErrorState.UNQUOTED_CHARS_AFTER_QUOTED_STRING); - failures.put("\"unterminated ", FieldParser.ParseErrorState.UNTERMINATED_QUOTED_STRING); - - for (Map.Entry failure : failures.entrySet()) { - int result = - stringParser.parseField( - failure.getKey().getBytes(ConfigConstants.DEFAULT_CHARSET), - 0, - failure.getKey().length(), - new byte[] {(byte) '|'}, - null); - assertThat(result).isEqualTo(-1); - assertThat(stringParser.getErrorState()).isEqualTo(failure.getValue()); - } - } - - @Test - @Disabled("Test disabled because we do not support double-quote escaped quotes right now.") - void testParserCorrectness() throws Exception { - // RFC 4180 Compliance Test content - // Taken from http://en.wikipedia.org/wiki/Comma-separated_values#Example - String fileContent = - "Year,Make,Model,Description,Price\n" - + "1997,Ford,E350,\"ac, abs, moon\",3000.00\n" - + "1999,Chevy,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00\n" - + "1996,Jeep,Grand Cherokee,\"MUST SELL! air, moon roof, loaded\",4799.00\n" - + "1999,Chevy,\"Venture \"\"Extended Edition, Very Large\"\"\",,5000.00\n" - + ",,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes); - format.setSkipFirstLineAsHeader(true); - format.setFieldDelimiter(","); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(5); - Row r1 = new Row(5); - r1.setField(0, 1997); - r1.setField(1, "Ford"); - r1.setField(2, "E350"); - r1.setField(3, "ac, abs, moon"); - r1.setField(4, 3000.0); - - Row r2 = new Row(5); - r2.setField(0, 1999); - r2.setField(1, "Chevy"); - r2.setField(2, "Venture \"Extended Edition\""); - r2.setField(3, ""); - r2.setField(4, 4900.0); - - Row r3 = new Row(5); - r3.setField(0, 1996); - r3.setField(1, "Jeep"); - r3.setField(2, "Grand Cherokee"); - r3.setField(3, "MUST SELL! air, moon roof, loaded"); - r3.setField(4, 4799.0); - - Row r4 = new Row(5); - r4.setField(0, 1999); - r4.setField(1, "Chevy"); - r4.setField(2, "Venture \"Extended Edition, Very Large\""); - r4.setField(3, ""); - r4.setField(4, 5000.0); - - Row r5 = new Row(5); - r5.setField(0, 0); - r5.setField(1, ""); - r5.setField(2, "Venture \"Extended Edition\""); - r5.setField(3, ""); - r5.setField(4, 4900.0); - - Row[] expectedLines = new Row[] {r1, r2, r3, r4, r5}; - for (Row expected : expectedLines) { - result = format.nextRecord(result); - assertThat(result).isEqualTo(expected); - } - assertThat(format.nextRecord(result)).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testWindowsLineEndRemoval() throws Exception { - - // check typical use case -- linux file is correct and it is set up to linux(\n) - testRemovingTrailingCR("\n", "\n"); - - // check typical windows case -- windows file endings and file has windows file endings set - // up - testRemovingTrailingCR("\r\n", "\r\n"); - - // check problematic case windows file -- windows file endings(\r\n) - // but linux line endings (\n) set up - testRemovingTrailingCR("\r\n", "\n"); - - // check problematic case linux file -- linux file endings (\n) - // but windows file endings set up (\r\n) - // specific setup for windows line endings will expect \r\n because - // it has to be set up and is not standard. - } - - @Test - void testQuotedStringParsingWithIncludeFields() throws Exception { - String fileContent = - "\"20:41:52-1-3-2015\"|\"Re: Taskmanager memory error in Eclipse\"|" - + "\"Blahblah \"|\"blaaa|\"blubb\""; - File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); - writer.write(fileContent); - writer.close(); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat inputFormat = - new RowCsvInputFormat( - new Path(tempFile.toURI().toString()), fieldTypes, new int[] {0, 2}); - inputFormat.enableQuotedStringParsing('"'); - inputFormat.setFieldDelimiter("|"); - inputFormat.setDelimiter('\n'); - inputFormat.configure(new Configuration()); - - FileInputSplit[] splits = inputFormat.createInputSplits(1); - inputFormat.open(splits[0]); - - Row record = inputFormat.nextRecord(new Row(2)); - assertThat(record.getField(0)).isEqualTo("20:41:52-1-3-2015"); - assertThat(record.getField(1)).isEqualTo("Blahblah "); - } - - @Test - void testQuotedStringParsingWithEscapedQuotes() throws Exception { - String fileContent = "\"\\\"Hello\\\" World\"|\"We are\\\" young\""; - File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); - writer.write(fileContent); - writer.close(); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO - }; - - RowCsvInputFormat inputFormat = - new RowCsvInputFormat(new Path(tempFile.toURI().toString()), fieldTypes); - inputFormat.enableQuotedStringParsing('"'); - inputFormat.setFieldDelimiter("|"); - inputFormat.setDelimiter('\n'); - inputFormat.configure(new Configuration()); - - FileInputSplit[] splits = inputFormat.createInputSplits(1); - inputFormat.open(splits[0]); - - Row record = inputFormat.nextRecord(new Row(2)); - assertThat(record.getField(0)).isEqualTo("\\\"Hello\\\" World"); - assertThat(record.getField(1)).isEqualTo("We are\\\" young"); - } - - @Test - void testSqlTimeFields() throws Exception { - String fileContent = - "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5\n" - + "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5.3\n"; - - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - SqlTimeTypeInfo.DATE, - SqlTimeTypeInfo.TIME, - SqlTimeTypeInfo.TIMESTAMP, - SqlTimeTypeInfo.TIMESTAMP - }; - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes); - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(4); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(Date.valueOf("1990-10-14")); - assertThat(result.getField(1)).isEqualTo(Time.valueOf("02:42:25")); - assertThat(result.getField(2)).isEqualTo(Timestamp.valueOf("1990-10-14 02:42:25.123")); - assertThat(result.getField(3)).isEqualTo(Timestamp.valueOf("1990-01-04 02:02:05")); - - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(Date.valueOf("1990-10-14")); - assertThat(result.getField(1)).isEqualTo(Time.valueOf("02:42:25")); - assertThat(result.getField(2)).isEqualTo(Timestamp.valueOf("1990-10-14 02:42:25.123")); - assertThat(result.getField(3)).isEqualTo(Timestamp.valueOf("1990-01-04 02:02:05.3")); - - result = format.nextRecord(result); - assertThat(result).isNull(); - assertThat(format.reachedEnd()).isTrue(); - } - - @Test - void testScanOrder() throws Exception { - String fileContent = - // first row - "111|222|333|444|555|666|777|888|999|000|\n" - + - // second row - "000|999|888|777|666|555|444|333|222|111|"; - FileInputSplit split = createTempFile(fileContent); - - TypeInformation[] fieldTypes = - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO - }; - - int[] order = new int[] {7, 3, 0}; - RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, order); - - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(3); - - // check first row - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(888); - assertThat(result.getField(1)).isEqualTo(444); - assertThat(result.getField(2)).isEqualTo(111); - - // check second row - result = format.nextRecord(result); - assertThat(result).isNotNull(); - assertThat(result.getField(0)).isEqualTo(333); - assertThat(result.getField(1)).isEqualTo(777); - assertThat(result.getField(2)).isEqualTo(0); - } - - @Test - void testEmptyProjection() throws Exception { - String fileContent = "111|222|333\n" + "000|999|888"; - FileInputSplit split = createTempFile(fileContent); - - RowCsvInputFormat format = new RowCsvInputFormat(PATH, new TypeInformation[0], new int[0]); - - format.setFieldDelimiter("|"); - format.configure(new Configuration()); - format.open(split); - - Row result = new Row(0); - - // check first row - result = format.nextRecord(result); - assertThat(result).isNotNull(); - - // check second row - result = format.nextRecord(result); - assertThat(result).isNotNull(); - } - - private static FileInputSplit createTempFile(String content) throws IOException { - File tempFile = File.createTempFile("test_contents", "tmp"); - tempFile.deleteOnExit(); - OutputStreamWriter wrt = - new OutputStreamWriter(new FileOutputStream(tempFile), StandardCharsets.UTF_8); - wrt.write(content); - wrt.close(); - return new FileInputSplit( - 0, - new Path(tempFile.toURI().toString()), - 0, - tempFile.length(), - new String[] {"localhost"}); - } - - private static void testRemovingTrailingCR(String lineBreakerInFile, String lineBreakerSetup) - throws IOException { - String fileContent = FIRST_PART + lineBreakerInFile + SECOND_PART + lineBreakerInFile; - - // create input file - File tempFile = File.createTempFile("CsvInputFormatTest", "tmp"); - tempFile.deleteOnExit(); - tempFile.setWritable(true); - - OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); - wrt.write(fileContent); - wrt.close(); - - TypeInformation[] fieldTypes = new TypeInformation[] {BasicTypeInfo.STRING_TYPE_INFO}; - - RowCsvInputFormat inputFormat = - new RowCsvInputFormat(new Path(tempFile.toURI().toString()), fieldTypes); - inputFormat.configure(new Configuration()); - inputFormat.setDelimiter(lineBreakerSetup); - - FileInputSplit[] splits = inputFormat.createInputSplits(1); - inputFormat.open(splits[0]); - - Row result = inputFormat.nextRecord(new Row(1)); - assertThat(result).as("Expecting to not return null").isNotNull(); - assertThat(result.getField(0)).isEqualTo(FIRST_PART); - - result = inputFormat.nextRecord(result); - assertThat(result).as("Expecting to not return null").isNotNull(); - assertThat(result.getField(0)).isEqualTo(SECOND_PART); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java deleted file mode 100644 index 378bc7116cd19..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java +++ /dev/null @@ -1,119 +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.api.java.io; - -import org.apache.flink.api.common.io.BinaryInputFormat; -import org.apache.flink.api.common.io.BinaryOutputFormat; -import org.apache.flink.api.common.io.BlockInfo; -import org.apache.flink.api.common.io.FirstAttemptInitializationContext; -import org.apache.flink.api.common.io.SequentialFormatTestBase; -import org.apache.flink.api.common.serialization.SerializerConfigImpl; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; -import org.apache.flink.core.memory.DataOutputView; - -import org.junit.jupiter.api.BeforeEach; - -import java.io.IOException; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for type serialization format. */ -class TypeSerializerFormatTest extends SequentialFormatTestBase> { - - TypeInformation> resultType; - - private final TypeSerializer> serializer; - - private BlockInfo block; - - TypeSerializerFormatTest() { - - resultType = TypeExtractor.getForObject(getRecord(0)); - - serializer = resultType.createSerializer(new SerializerConfigImpl()); - } - - @BeforeEach - void setup() { - block = createInputFormat().createBlockInfo(); - } - - @Override - protected BinaryInputFormat> createInputFormat() { - Configuration configuration = new Configuration(); - - final TypeSerializerInputFormat> inputFormat = - new TypeSerializerInputFormat<>(resultType); - inputFormat.setFilePath(this.tempFile.toURI().toString()); - - inputFormat.configure(configuration); - return inputFormat; - } - - @Override - protected BinaryOutputFormat> createOutputFormat( - String path, Configuration configuration) throws IOException { - TypeSerializerOutputFormat> outputFormat = - new TypeSerializerOutputFormat<>(); - - outputFormat.setSerializer(serializer); - outputFormat.setOutputFilePath(new Path(path)); - outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE); - - configuration = configuration == null ? new Configuration() : configuration; - - outputFormat.configure(configuration); - outputFormat.open(FirstAttemptInitializationContext.of(0, 1)); - - return outputFormat; - } - - @Override - protected int getInfoSize() { - return block.getInfoSize(); - } - - @Override - protected Tuple2 getRecord(int index) { - return new Tuple2<>(index, String.valueOf(index)); - } - - @Override - protected Tuple2 createInstance() { - return new Tuple2<>(); - } - - @Override - protected void writeRecord(Tuple2 record, DataOutputView outputView) - throws IOException { - serializer.serialize(record, outputView); - } - - @Override - protected void checkEquals(Tuple2 expected, Tuple2 actual) { - assertThat(actual.f0).isEqualTo(expected.f0); - assertThat(actual.f1).isEqualTo(expected.f1); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java deleted file mode 100644 index a2a78fce3a869..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java +++ /dev/null @@ -1,101 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -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.aggregation.UnsupportedAggregationTypeException; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#aggregate(Aggregations, int)}. */ -class AggregateOperatorTest { - - // TUPLE DATA - - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - // LONG DATA - - private final List emptyLongData = new ArrayList<>(); - - @Test - void testFieldsAggregate() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.aggregate(Aggregations.SUM, 1); - - // should not work: index out of bounds - assertThatThrownBy(() -> tupleDs.aggregate(Aggregations.SUM, 10)) - .isInstanceOf(IllegalArgumentException.class); - - // should not work: not applied to tuple dataset - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - - assertThatThrownBy(() -> longDs.aggregate(Aggregations.MIN, 1)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testAggregationTypes() { - try { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work: multiple aggregates - tupleDs.aggregate(Aggregations.SUM, 0).and(Aggregations.MIN, 4); - - // should work: nested aggregates - tupleDs.aggregate(Aggregations.MIN, 2).aggregate(Aggregations.SUM, 1); - - // should not work: average on string - assertThatThrownBy(() -> tupleDs.aggregate(Aggregations.SUM, 2)) - .isInstanceOf(UnsupportedAggregationTypeException.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java deleted file mode 100644 index f755066a7457f..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java +++ /dev/null @@ -1,553 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operator.JoinOperatorTest.CustomType; -import org.apache.flink.api.java.operators.CoGroupOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#coGroup(DataSet)}. */ -@SuppressWarnings("serial") -class CoGroupOperatorTest { - - // TUPLE DATA - private static final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private static final List customTypeData = new ArrayList<>(); - - @BeforeAll - static void insertCustomData() { - customTypeData.add(new CustomType()); - } - - @Test - void testCoGroupKeyFields1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.coGroup(ds2).where(0).equalTo(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCoGroupKeyFields2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, incompatible cogroup key types - assertThatThrownBy(() -> ds1.coGroup(ds2).where(0).equalTo(2)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyFields3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, incompatible number of cogroup keys - assertThatThrownBy(() -> ds1.coGroup(ds2).where(0, 1).equalTo(2)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyFields4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, cogroup key out of range - assertThatThrownBy(() -> ds1.coGroup(ds2).where(5).equalTo(0)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCoGroupKeyFields5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, negative key field position - assertThatThrownBy(() -> ds1.coGroup(ds2).where(-1).equalTo(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCoGroupKeyFields6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, cogroup key fields on custom type - assertThatThrownBy(() -> ds1.coGroup(ds2).where(4).equalTo(0)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyExpressions1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.coGroup(ds2).where("myInt").equalTo("myInt"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCoGroupKeyExpressions2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible cogroup key types - assertThatThrownBy(() -> ds1.coGroup(ds2).where("myInt").equalTo("myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyExpressions3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible number of cogroup keys - assertThatThrownBy(() -> ds1.coGroup(ds2).where("myInt", "myString").equalTo("myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyExpressions4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, cogroup key non-existent - assertThatThrownBy(() -> ds1.coGroup(ds2).where("myNonExistent").equalTo("myInt")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testCoGroupKeyAtomicExpression1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromElements(0, 0, 1); - - ds1.coGroup(ds2).where("myInt").equalTo("*"); - } - - @Test - void testCoGroupKeyAtomicExpression2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 1); - DataSet ds2 = env.fromCollection(customTypeData); - - ds1.coGroup(ds2).where("*").equalTo("myInt"); - } - - @Test - void testCoGroupKeyAtomicInvalidExpression1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 1); - DataSet ds2 = env.fromCollection(customTypeData); - assertThatThrownBy(() -> ds1.coGroup(ds2).where("*", "invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyAtomicInvalidExpression2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 1); - DataSet ds2 = env.fromCollection(customTypeData); - - assertThatThrownBy(() -> ds1.coGroup(ds2).where("invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyAtomicInvalidExpression3() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromElements(0, 0, 1); - - assertThatThrownBy(() -> ds1.coGroup(ds2).where("myInt").equalTo("invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyAtomicInvalidExpression4() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromElements(0, 0, 1); - - assertThatThrownBy(() -> ds1.coGroup(ds2).where("myInt").equalTo("*", "invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyAtomicInvalidExpression5() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = env.fromElements(new ArrayList<>()); - DataSet ds2 = env.fromElements(0, 0, 0); - - assertThatThrownBy(() -> ds1.coGroup(ds2).where("*")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyAtomicInvalidExpression6() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 0); - DataSet> ds2 = env.fromElements(new ArrayList<>()); - - assertThatThrownBy(() -> ds1.coGroup(ds2).where("*").equalTo("*")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyExpressions1Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.coGroup(ds2).where("nested.myInt").equalTo("nested.myInt"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testCoGroupKeyExpressions2Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible cogroup key types - assertThatThrownBy(() -> ds1.coGroup(ds2).where("nested.myInt").equalTo("nested.myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyExpressions3Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible number of cogroup keys - assertThatThrownBy( - () -> - ds1.coGroup(ds2) - .where("nested.myInt", "nested.myString") - .equalTo("nested.myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyExpressions4Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, cogroup key non-existent - assertThatThrownBy( - () -> - ds1.coGroup(ds2) - .where("nested.myNonExistent") - .equalTo("nested.myInt")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testCoGroupKeySelectors1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.coGroup(ds2) - .where((KeySelector) value -> value.myLong) - .equalTo((KeySelector) value -> value.myLong); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCoGroupKeyMixing1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.coGroup(ds2) - .where((KeySelector) value -> value.myLong) - .equalTo(3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCoGroupKeyMixing2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.coGroup(ds2) - .where(3) - .equalTo((KeySelector) value -> value.myLong); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCoGroupKeyMixing3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible types - assertThatThrownBy( - () -> - ds1.coGroup(ds2) - .where(2) - .equalTo( - (KeySelector) - value -> value.myLong)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testCoGroupKeyMixing4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, more than one key field position - assertThatThrownBy( - () -> - ds1.coGroup(ds2) - .where(1, 3) - .equalTo( - (KeySelector) - value -> value.myLong)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSemanticPropsWithKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> tupleDs2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - CoGroupOperator coGroupOp = - tupleDs1.coGroup(tupleDs2) - .where(new DummyTestKeySelector()) - .equalTo(new DummyTestKeySelector()) - .with(new DummyTestCoGroupFunction1()); - - SemanticProperties semProps = coGroupOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(1, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 6)).containsExactly(0); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 4, 6); - assertThat(semProps.getReadFields(1)).containsExactly(5, 3); - } - - @Test - void testSemanticPropsWithKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> tupleDs2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - CoGroupOperator coGroupOp = - tupleDs1.coGroup(tupleDs2) - .where(new DummyTestKeySelector()) - .equalTo(new DummyTestKeySelector()) - .with(new DummyTestCoGroupFunction2()) - .withForwardedFieldsFirst("2;4->0") - .withForwardedFieldsSecond("0->4;1;1->3"); - - SemanticProperties semProps = coGroupOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(0); - - assertThat(semProps.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(1, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(1, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 6)).isEmpty(); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 3, 4); - - assertThat(semProps.getReadFields(1)).isNull(); - } - - private static class DummyTestKeySelector - implements KeySelector< - Tuple5, Tuple2> { - @Override - public Tuple2 getKey(Tuple5 value) - throws Exception { - return new Tuple2<>(); - } - } - - @FunctionAnnotation.ForwardedFieldsFirst("0->4;1;1->3") - @FunctionAnnotation.ForwardedFieldsSecond("2;4->0") - @FunctionAnnotation.ReadFieldsFirst("0;2;4") - @FunctionAnnotation.ReadFieldsSecond("1;3") - private static class DummyTestCoGroupFunction1 - implements CoGroupFunction< - Tuple5, - Tuple5, - Tuple5> { - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception {} - } - - @FunctionAnnotation.ReadFieldsFirst("0;1;2") - private static class DummyTestCoGroupFunction2 - implements CoGroupFunction< - Tuple5, - Tuple5, - Tuple5> { - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception {} - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java deleted file mode 100644 index 17f9f4883be4d..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java +++ /dev/null @@ -1,478 +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.api.java.operator; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#cross(DataSet)}. */ -class CrossOperatorTest { - - // TUPLE DATA - private static final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private static final List customTypeData = new ArrayList<>(); - - @BeforeAll - static void insertCustomData() { - customTypeData.add(new CustomType()); - } - - @Test - void testCrossProjection1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection21() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0, 3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection22() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0, 3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0).projectSecond(3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection23() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0).projectSecond(3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0, 2).projectSecond(1, 4).projectFirst(1); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection24() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectFirst(0, 2).projectSecond(1, 4).projectFirst(1); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectSecond(0, 2).projectFirst(1, 4).projectFirst(1); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection25() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectSecond(0, 2).projectFirst(1, 4).projectFirst(1); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.cross(ds2).projectFirst().projectSecond(); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection26() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.cross(ds2).projectFirst().projectSecond(); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection7() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectSecond().projectFirst(1, 4); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection27() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.cross(ds2).projectSecond().projectFirst(1, 4); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testCrossProjection8() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.cross(ds2).projectFirst(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCrossProjection28() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.cross(ds2).projectFirst(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCrossProjection9() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.cross(ds2).projectSecond(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCrossProjection29() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.cross(ds2).projectSecond(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCrossProjection10() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.cross(ds2).projectFirst(2); - } - - @Test - void testCrossProjection30() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, type does not match - assertThatThrownBy(() -> ds1.cross(ds2).projectFirst(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - void testCrossProjection11() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.cross(ds2).projectSecond(2); - } - - @Test - void testCrossProjection31() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, type does not match - assertThatThrownBy(() -> ds1.cross(ds2).projectSecond(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - void testCrossProjection12() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.cross(ds2).projectSecond(2).projectFirst(1); - } - - @Test - void testCrossProjection32() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, number of types and fields does not match - assertThatThrownBy(() -> ds1.cross(ds2).projectSecond(2).projectFirst(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCrossProjection13() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.cross(ds2).projectSecond(0).projectFirst(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testCrossProjection14() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.cross(ds2).projectFirst(0).projectSecond(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - /** Custom data type, for testing purposes. */ - static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java deleted file mode 100644 index ea6a0cdb64291..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java +++ /dev/null @@ -1,356 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.CompositeType; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#writeAsText(String)}. */ -class DataSinkTest { - - // TUPLE DATA - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - // POJO DATA - private final List pojoData = new ArrayList<>(); - - @BeforeEach - void fillPojoData() { - if (pojoData.isEmpty()) { - pojoData.add(new CustomType()); - } - } - - @Test - void testTupleSingleOrderIdx() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.writeAsText("/tmp/willNotHappen").sortLocalOutput(0, Order.ANY); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testTupleTwoOrderIdx() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput(0, Order.ASCENDING) - .sortLocalOutput(3, Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testTupleSingleOrderExp() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.writeAsText("/tmp/willNotHappen").sortLocalOutput("f0", Order.ANY); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - void testTupleSingleOrderExpFull() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work - tupleDs.writeAsText("/tmp/willNotHappen").sortLocalOutput("*", Order.ANY); - } - - @Test - void testTupleTwoOrderExp() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("f1", Order.ASCENDING) - .sortLocalOutput("f4", Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testTupleTwoOrderMixed() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput(4, Order.ASCENDING) - .sortLocalOutput("f2", Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testFailTupleIndexOutOfBounds() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // must not work - assertThatThrownBy( - () -> - tupleDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput(3, Order.ASCENDING) - .sortLocalOutput(5, Order.DESCENDING)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testFailTupleInv() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // must not work - assertThatThrownBy( - () -> - tupleDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("notThere", Order.ASCENDING) - .sortLocalOutput("f4", Order.DESCENDING)) - .isInstanceOf(CompositeType.InvalidFieldReferenceException.class); - } - - @Test - void testPrimitiveOrder() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet longDs = env.generateSequence(0, 2); - - // should work - try { - longDs.writeAsText("/tmp/willNotHappen").sortLocalOutput("*", Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testFailPrimitiveOrder1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet longDs = env.generateSequence(0, 2); - - // must not work - assertThatThrownBy( - () -> - longDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput(0, Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testFailPrimitiveOrder2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet longDs = env.generateSequence(0, 2); - - // must not work - assertThatThrownBy( - () -> - longDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("0", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testFailPrimitiveOrder3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet longDs = env.generateSequence(0, 2); - - // must not work - assertThatThrownBy( - () -> - longDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("nope", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testPojoSingleOrder() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet pojoDs = env.fromCollection(pojoData); - - // should work - try { - pojoDs.writeAsText("/tmp/willNotHappen").sortLocalOutput("myString", Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testPojoTwoOrder() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet pojoDs = env.fromCollection(pojoData); - - // should work - try { - pojoDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("myLong", Order.ASCENDING) - .sortLocalOutput("myString", Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testFailPojoIdx() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet pojoDs = env.fromCollection(pojoData); - - // must not work - assertThatThrownBy( - () -> - pojoDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput(1, Order.DESCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testFailPojoInvalidField() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet pojoDs = env.fromCollection(pojoData); - - // must not work - assertThatThrownBy( - () -> - pojoDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("myInt", Order.ASCENDING) - .sortLocalOutput("notThere", Order.DESCENDING)) - .isInstanceOf(CompositeType.InvalidFieldReferenceException.class); - } - - @Test - void testPojoSingleOrderFull() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet pojoDs = env.fromCollection(pojoData); - - // must not work - assertThatThrownBy( - () -> - pojoDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("*", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testArrayOrderFull() { - - List arrayData = new ArrayList<>(); - arrayData.add(new Object[0]); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet pojoDs = env.fromCollection(arrayData); - - // must not work - assertThatThrownBy( - () -> - pojoDs.writeAsText("/tmp/willNotHappen") - .sortLocalOutput("*", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java deleted file mode 100644 index a9819feef9ac3..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java +++ /dev/null @@ -1,243 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -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.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#distinct()}. */ -class DistinctOperatorTest { - - // TUPLE DATA - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - // LONG DATA - private final List emptyLongData = new ArrayList<>(); - - private final List customTypeData = new ArrayList<>(); - - @Test - void testDistinctByKeyFields1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.distinct(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testDistinctByKeyFields2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - // should not work: distinct on basic type - assertThatThrownBy(() -> longDs.distinct(0)).isInstanceOf(InvalidProgramException.class); - } - - @Test - void testDistinctByKeyFields3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - // should not work: distinct on custom type - assertThatThrownBy(() -> customDs.distinct(0)).isInstanceOf(InvalidProgramException.class); - } - - @Test - void testDistinctByKeyFields4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.distinct(); - } - - @Test - void testDistinctByKeyFields5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - - // should work - customDs.distinct(); - } - - @Test - void testDistinctByKeyFields6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, negative field position - assertThatThrownBy(() -> tupleDs.distinct(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testDistinctByKeyFields7() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - - // should work - try { - longDs.distinct("*"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - @SuppressWarnings("serial") - void testDistinctByKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - this.customTypeData.add(new CustomType()); - - try { - DataSet customDs = env.fromCollection(customTypeData); - // should work - customDs.distinct( - new KeySelector() { - - @Override - public Long getKey(CustomType value) { - return value.myLong; - } - }); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testDistinctByKeyIndices1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - try { - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - // should work - longDs.distinct(); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testDistinctOnNotKeyDataType() { - /* - * should not work. NotComparable data type cannot be used as key - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - NotComparable a = new NotComparable(); - List l = new ArrayList<>(); - l.add(a); - - DataSet ds = env.fromCollection(l); - assertThatThrownBy( - () -> { - DataSet reduceDs = ds.distinct(); - }) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testDistinctOnNotKeyDataTypeOnSelectAllChar() { - /* - * should not work. NotComparable data type cannot be used as key - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - NotComparable a = new NotComparable(); - List l = new ArrayList<>(); - l.add(a); - - DataSet ds = env.fromCollection(l); - assertThatThrownBy( - () -> { - DataSet reduceDs = ds.distinct("*"); - }) - .isInstanceOf(InvalidProgramException.class); - } - - static class NotComparable { - public List myInts; - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java deleted file mode 100644 index 22c93cb5613f5..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DataSet#first(int)}. */ -class FirstNOperatorTest { - - // TUPLE DATA - - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testUngroupedFirstN() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.first(1); - - // should work - tupleDs.first(10); - - // should not work n == 0 - assertThatThrownBy(() -> tupleDs.first(0)).isInstanceOf(InvalidProgramException.class); - - // should not work n == -1 - assertThatThrownBy(() -> tupleDs.first(-1)).isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupedFirstN() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.groupBy(2).first(1); - - // should work - tupleDs.groupBy(1, 3).first(10); - - // should not work n == 0 - assertThatThrownBy(() -> tupleDs.groupBy(0).first(0)) - .isInstanceOf(InvalidProgramException.class); - - // should not work n == -1 - assertThatThrownBy(() -> tupleDs.groupBy(2).first(-1)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupedSortedFirstN() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.groupBy(2).sortGroup(4, Order.ASCENDING).first(1); - - // should work - tupleDs.groupBy(1, 3).sortGroup(4, Order.ASCENDING).first(10); - - // should not work n == 0 - assertThatThrownBy(() -> tupleDs.groupBy(0).sortGroup(4, Order.ASCENDING).first(0)) - .isInstanceOf(InvalidProgramException.class); - - // should not work n == -1 - assertThatThrownBy(() -> tupleDs.groupBy(2).sortGroup(4, Order.ASCENDING).first(-1)) - .isInstanceOf(InvalidProgramException.class); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java deleted file mode 100644 index 33cec421d7968..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java +++ /dev/null @@ -1,273 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.CompositeType; -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.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DataSet#fullOuterJoin(DataSet)}. */ -class FullOuterJoinOperatorTest { - - // TUPLE DATA - private static final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testFullOuter1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2).where(0).equalTo(4).with(new DummyJoin()); - } - - @Test - void testFullOuter2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2).where("f1").equalTo("f3").with(new DummyJoin()); - } - - @Test - void testFullOuter3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2) - .where(new IntKeySelector()) - .equalTo(new IntKeySelector()) - .with(new DummyJoin()); - } - - @Test - void testFullOuter4() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2).where(0).equalTo(new IntKeySelector()).with(new DummyJoin()); - } - - @Test - void testFullOuter5() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2).where(new IntKeySelector()).equalTo("f4").with(new DummyJoin()); - } - - @Test - void testFullOuter6() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2).where("f0").equalTo(4).with(new DummyJoin()); - } - - @Test - void testFullOuter7() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // invalid key position - assertThatThrownBy(() -> ds1.fullOuterJoin(ds2).where(5).equalTo(0).with(new DummyJoin())) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testFullOuter8() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // invalid key reference - assertThatThrownBy( - () -> ds1.fullOuterJoin(ds2).where(1).equalTo("f5").with(new DummyJoin())) - .isInstanceOf(CompositeType.InvalidFieldReferenceException.class); - } - - @Test - void testFullOuter9() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // key types do not match - assertThatThrownBy(() -> ds1.fullOuterJoin(ds2).where(0).equalTo(1).with(new DummyJoin())) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testFullOuter10() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // key types do not match - assertThatThrownBy( - () -> - ds1.fullOuterJoin(ds2) - .where(new IntKeySelector()) - .equalTo(new LongKeySelector()) - .with(new DummyJoin())) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testFullOuterStrategy1() { - this.testFullOuterStrategies(JoinHint.OPTIMIZER_CHOOSES); - } - - @Test - void testFullOuterStrategy2() { - this.testFullOuterStrategies(JoinHint.REPARTITION_SORT_MERGE); - } - - @Test - void testFullOuterStrategy3() { - this.testFullOuterStrategies(JoinHint.REPARTITION_HASH_SECOND); - } - - @Test - void testFullOuterStrategy4() { - assertThatThrownBy(() -> this.testFullOuterStrategies(JoinHint.BROADCAST_HASH_SECOND)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testFullOuterStrategy5() { - this.testFullOuterStrategies(JoinHint.REPARTITION_HASH_FIRST); - } - - @Test - void testFullOuterStrategy6() { - assertThatThrownBy(() -> this.testFullOuterStrategies(JoinHint.BROADCAST_HASH_FIRST)) - .isInstanceOf(InvalidProgramException.class); - } - - private void testFullOuterStrategies(JoinHint hint) { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.fullOuterJoin(ds2, hint).where(0).equalTo(4).with(new DummyJoin()); - } - - /* - * #################################################################### - */ - - @SuppressWarnings("serial") - private static class DummyJoin - implements JoinFunction< - Tuple5, - Tuple5, - Long> { - - @Override - public Long join( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return 1L; - } - } - - @SuppressWarnings("serial") - private static class IntKeySelector - implements KeySelector, Integer> { - - @Override - public Integer getKey(Tuple5 v) throws Exception { - return v.f0; - } - } - - @SuppressWarnings("serial") - private static class LongKeySelector - implements KeySelector, Long> { - - @Override - public Long getKey(Tuple5 v) throws Exception { - return v.f1; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java deleted file mode 100644 index e98e705f0de8c..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java +++ /dev/null @@ -1,358 +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.api.java.operator; - -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.GroupCombineOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link DataSet#combineGroup(GroupCombineFunction)}. */ -@SuppressWarnings("serial") -class GroupCombineOperatorTest { - - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testSemanticPropsWithKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .combineGroup(new DummyGroupCombineFunction1()); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(2); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 5, 6); - } - - @Test - void testSemanticPropsWithKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .sortGroup(new DummyTestKeySelector(), Order.ASCENDING) - .combineGroup(new DummyGroupCombineFunction1()); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 7)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 8)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(4); - - assertThat(semProps.getReadFields(0)).containsExactly(8, 4, 7); - } - - @Test - void testSemanticPropsWithKeySelector3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .combineGroup(new DummyGroupCombineFunction2()) - .withForwardedFields("0->4;1;1->3;2"); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(2); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 5, 6); - } - - @Test - void testSemanticPropsWithKeySelector4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .sortGroup(new DummyTestKeySelector(), Order.ASCENDING) - .combineGroup(new DummyGroupCombineFunction2()) - .withForwardedFields("0->4;1;1->3;2"); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 7)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 8)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(4); - - assertThat(semProps.getReadFields(0)).containsExactly(8, 4, 7); - } - - @Test - void testSemanticPropsWithKeySelector5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .combineGroup(new DummyGroupCombineFunction3()) - .withForwardedFields("4->0;3;3->1;2"); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(0); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - @Test - void testSemanticPropsWithKeySelector6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .sortGroup(new DummyTestKeySelector(), Order.ASCENDING) - .combineGroup(new DummyGroupCombineFunction3()) - .withForwardedFields("4->0;3;3->1;2"); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 7)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 8)).containsExactly(0); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(8); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(7); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(7); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - @Test - void testSemanticPropsWithKeySelector7() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupCombineOperator< - Tuple5, - Tuple5> - combineOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .combineGroup(new DummyGroupCombineFunction4()); - - SemanticProperties semProps = combineOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(3); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(2); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - private static class DummyTestKeySelector - implements KeySelector< - Tuple5, Tuple2> { - @Override - public Tuple2 getKey(Tuple5 value) - throws Exception { - return new Tuple2<>(); - } - } - - @FunctionAnnotation.ForwardedFields("0->4;1;1->3;2") - @FunctionAnnotation.ReadFields("0;3;4") - private static class DummyGroupCombineFunction1 - implements GroupCombineFunction< - Tuple5, - Tuple5> { - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception {} - } - - @FunctionAnnotation.ReadFields("0;3;4") - private static class DummyGroupCombineFunction2 - implements GroupCombineFunction< - Tuple5, - Tuple5> { - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception {} - } - - private static class DummyGroupCombineFunction3 - implements GroupCombineFunction< - Tuple5, - Tuple5> { - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception {} - } - - @FunctionAnnotation.NonForwardedFields("2;4") - private static class DummyGroupCombineFunction4 - implements GroupCombineFunction< - Tuple5, - Tuple5> { - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception {} - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java deleted file mode 100644 index 78c94be9bd618..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java +++ /dev/null @@ -1,358 +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.api.java.operator; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.GroupReduceOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link DataSet#reduceGroup(GroupReduceFunction)}. */ -@SuppressWarnings("serial") -class GroupReduceOperatorTest { - - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testSemanticPropsWithKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .reduceGroup(new DummyGroupReduceFunction1()); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(2); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 5, 6); - } - - @Test - void testSemanticPropsWithKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .sortGroup(new DummyTestKeySelector(), Order.ASCENDING) - .reduceGroup(new DummyGroupReduceFunction1()); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 7)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 8)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(4); - - assertThat(semProps.getReadFields(0)).containsExactly(8, 4, 7); - } - - @Test - void testSemanticPropsWithKeySelector3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .reduceGroup(new DummyGroupReduceFunction2()) - .withForwardedFields("0->4;1;1->3;2"); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(2); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 5, 6); - } - - @Test - void testSemanticPropsWithKeySelector4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .sortGroup(new DummyTestKeySelector(), Order.ASCENDING) - .reduceGroup(new DummyGroupReduceFunction2()) - .withForwardedFields("0->4;1;1->3;2"); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 7)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 8)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(4); - - assertThat(semProps.getReadFields(0)).containsExactly(8, 4, 7); - } - - @Test - void testSemanticPropsWithKeySelector5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .reduceGroup(new DummyGroupReduceFunction3()) - .withForwardedFields("4->0;3;3->1;2"); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(0); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - @Test - void testSemanticPropsWithKeySelector6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .sortGroup(new DummyTestKeySelector(), Order.ASCENDING) - .reduceGroup(new DummyGroupReduceFunction3()) - .withForwardedFields("4->0;3;3->1;2"); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 7)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 8)).containsExactly(0); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(8); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(7); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(7); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - @Test - void testSemanticPropsWithKeySelector7() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - GroupReduceOperator< - Tuple5, - Tuple5> - reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .reduceGroup(new DummyGroupReduceFunction4()); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(3); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(2); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - private static class DummyTestKeySelector - implements KeySelector< - Tuple5, Tuple2> { - @Override - public Tuple2 getKey(Tuple5 value) - throws Exception { - return new Tuple2<>(); - } - } - - @FunctionAnnotation.ForwardedFields("0->4;1;1->3;2") - @FunctionAnnotation.ReadFields("0;3;4") - private static class DummyGroupReduceFunction1 - implements GroupReduceFunction< - Tuple5, - Tuple5> { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception {} - } - - @FunctionAnnotation.ReadFields("0;3;4") - private static class DummyGroupReduceFunction2 - implements GroupReduceFunction< - Tuple5, - Tuple5> { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception {} - } - - private static class DummyGroupReduceFunction3 - implements GroupReduceFunction< - Tuple5, - Tuple5> { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception {} - } - - @FunctionAnnotation.NonForwardedFields("2;4") - private static class DummyGroupReduceFunction4 - implements GroupReduceFunction< - Tuple5, - Tuple5> { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception {} - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java deleted file mode 100644 index 96cec516d464a..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java +++ /dev/null @@ -1,666 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -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.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#groupBy(int...)}. */ -class GroupingTest { - - // TUPLE DATA - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private final TupleTypeInfo> tupleWithCustomInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - TypeExtractor.createTypeInfo(CustomType.class), - BasicArrayTypeInfo.LONG_ARRAY_TYPE_INFO); - - // LONG DATA - private final List emptyLongData = new ArrayList<>(); - - private final List customTypeData = new ArrayList<>(); - - private final List> tupleWithCustomData = - new ArrayList<>(); - - private final List> byteArrayData = new ArrayList<>(); - - @Test - void testGroupByKeyFields1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.groupBy(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupByKeyFields2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - // should not work: groups on basic type - assertThatThrownBy(() -> longDs.groupBy(0)).isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupByKeyFields3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - // should not work: groups on custom type - assertThatThrownBy(() -> customDs.groupBy(0)).isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupByKeyFields4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.groupBy(5)).isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testGroupByKeyFields5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, negative field position - assertThatThrownBy(() -> tupleDs.groupBy(-1)).isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testGroupByKeyFieldsOnPrimitiveArray() { - this.byteArrayData.add(new Tuple2<>(new byte[] {0}, new byte[] {1})); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = env.fromCollection(byteArrayData); - tupleDs.groupBy(0); - } - - @Test - void testGroupByKeyExpressions1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet ds = env.fromCollection(customTypeData); - - // should work - try { - ds.groupBy("myInt"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupByKeyExpressions2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - // should not work: groups on basic type - assertThatThrownBy(() -> longDs.groupBy("myInt")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupByKeyExpressions3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - // should not work: tuple selector on custom type - assertThatThrownBy(() -> customDs.groupBy(0)).isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupByKeyExpressions4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // should not work, key out of tuple bounds - assertThatThrownBy( - () -> { - DataSet ds = env.fromCollection(customTypeData); - ds.groupBy("myNonExistent"); - }) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testGroupByKeyExpressions1Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet ds = env.fromCollection(customTypeData); - - // should work - try { - ds.groupBy("nested.myInt"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupByKeyExpressions2Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // should not work, key out of tuple bounds - assertThatThrownBy( - () -> { - DataSet ds = env.fromCollection(customTypeData); - ds.groupBy("nested.myNonExistent"); - }) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - @SuppressWarnings("serial") - void testGroupByKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - this.customTypeData.add(new CustomType()); - - try { - DataSet customDs = env.fromCollection(customTypeData); - // should work - customDs.groupBy((KeySelector) value -> value.myLong); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - @SuppressWarnings("serial") - void testGroupByKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - this.customTypeData.add(new CustomType()); - - try { - DataSet customDs = env.fromCollection(customTypeData); - // should work - customDs.groupBy( - new KeySelector>() { - @Override - public Tuple2 getKey(CustomType value) { - return new Tuple2(value.myInt, value.myLong); - } - }); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - @SuppressWarnings("serial") - void testGroupByKeySelector3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - this.customTypeData.add(new CustomType()); - - try { - DataSet customDs = env.fromCollection(customTypeData); - // should not work - customDs.groupBy( - new KeySelector() { - @Override - public CustomType getKey(CustomType value) { - return value; - } - }); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - @SuppressWarnings("serial") - void testGroupByKeySelector4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - this.customTypeData.add(new CustomType()); - - try { - DataSet customDs = env.fromCollection(customTypeData); - // should not work - customDs.groupBy( - new KeySelector< - GroupingTest.CustomType, Tuple2>() { - @Override - public Tuple2 getKey(CustomType value) { - return new Tuple2(value.myInt, value); - } - }); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - @SuppressWarnings("serial") - void testGroupByKeySelector5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - // should not work - assertThatThrownBy( - () -> - customDs.groupBy( - (KeySelector) - value -> new CustomType2())) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupSortKeyFields1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.groupBy(0).sortGroup(0, Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupSortKeyFields2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, field index out of bounds - assertThatThrownBy(() -> tupleDs.groupBy(0).sortGroup(5, Order.ASCENDING)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testGroupSortKeyFields3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - - // should not work: sorted groups on groupings by key selectors - assertThatThrownBy( - () -> - longDs.groupBy( - new KeySelector() { - private static final long serialVersionUID = 1L; - - @Override - public Long getKey(Long value) { - return value; - } - }) - .sortGroup(0, Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupSortKeyFields4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy(() -> tupleDs.groupBy(0).sortGroup(2, Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupSortKeyFields5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy(() -> tupleDs.groupBy(0).sortGroup(3, Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testChainedGroupSortKeyFields() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.groupBy(0).sortGroup(0, Order.ASCENDING).sortGroup(2, Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupSortByKeyExpression1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should work - try { - tupleDs.groupBy("f0").sortGroup("f1", Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupSortByKeyExpression2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should work - try { - tupleDs.groupBy("f0").sortGroup("f2.myString", Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupSortByKeyExpression3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should work - try { - tupleDs.groupBy("f0") - .sortGroup("f2.myString", Order.ASCENDING) - .sortGroup("f1", Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testGroupSortByKeyExpression4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy(() -> tupleDs.groupBy("f0").sortGroup("f2", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupSortByKeyExpression5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy( - () -> - tupleDs.groupBy("f0") - .sortGroup("f1", Order.ASCENDING) - .sortGroup("f2", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupSortByKeyExpression6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy(() -> tupleDs.groupBy("f0").sortGroup("f3", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @SuppressWarnings("serial") - @Test - void testGroupSortByKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - tupleDs.groupBy( - new KeySelector, Long>() { - @Override - public Long getKey(Tuple4 value) - throws Exception { - return value.f1; - } - }) - .sortGroup( - new KeySelector, Integer>() { - @Override - public Integer getKey(Tuple4 value) - throws Exception { - return value.f0; - } - }, - Order.ASCENDING); - } - - @SuppressWarnings("serial") - @Test - void testGroupSortByKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy( - () -> - tupleDs.groupBy( - (KeySelector< - Tuple4< - Integer, - Long, - CustomType, - Long[]>, - Long>) - value -> value.f1) - .sortGroup( - (KeySelector< - Tuple4< - Integer, - Long, - CustomType, - Long[]>, - CustomType>) - value -> value.f2, - Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @SuppressWarnings("serial") - @Test - void testGroupSortByKeySelector3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should not work - assertThatThrownBy( - () -> - tupleDs.groupBy( - (KeySelector< - Tuple4< - Integer, - Long, - CustomType, - Long[]>, - Long>) - value -> value.f1) - .sortGroup( - (KeySelector< - Tuple4< - Integer, - Long, - CustomType, - Long[]>, - Long[]>) - value -> value.f3, - Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupingAtomicType() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet dataSet = env.fromElements(0, 1, 1, 2, 0, 0); - - dataSet.groupBy("*"); - } - - @Test - void testGroupAtomicTypeWithInvalid1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet dataSet = env.fromElements(0, 1, 2, 3); - - assertThatThrownBy(() -> dataSet.groupBy("*", "invalidField")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupAtomicTypeWithInvalid2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet dataSet = env.fromElements(0, 1, 2, 3); - - assertThatThrownBy(() -> dataSet.groupBy("invalidField")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testGroupAtomicTypeWithInvalid3() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> dataSet = env.fromElements(new ArrayList()); - - assertThatThrownBy(() -> dataSet.groupBy("*")).isInstanceOf(InvalidProgramException.class); - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - /** Custom nested data type, for testing purposes. */ - public static class Nest { - public int myInt; - } - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - public Nest nested; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } - - /** Custom non-nested data type, for testing purposes. */ - public static class CustomType2 implements Serializable { - - public int myInt; - public Integer[] myIntArray; - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java deleted file mode 100644 index 27aedaa686163..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java +++ /dev/null @@ -1,1317 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.CompositeType; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.JoinOperator; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; - -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#join(DataSet)}. */ -@SuppressWarnings("serial") -class JoinOperatorTest { - - // TUPLE DATA - private static final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - // TUPLE DATA with nested Tuple2 - private static final List, Long, String, Long, Integer>> - emptyNestedTupleData = new ArrayList<>(); - - private final TupleTypeInfo, Long, String, Long, Integer>> - nestedTupleTypeInfo = - new TupleTypeInfo<>( - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - // TUPLE DATA with nested CustomType - private static final List> - emptyNestedCustomTupleData = new ArrayList<>(); - - private final TupleTypeInfo> - nestedCustomTupleTypeInfo = - new TupleTypeInfo<>( - TypeExtractor.getForClass(CustomType.class), - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private static final List customTypeWithTupleData = new ArrayList<>(); - private static final List customTypeData = new ArrayList<>(); - - private static final List customNestedTypeData = new ArrayList<>(); - - @BeforeAll - static void insertCustomData() { - customTypeData.add(new CustomType()); - customTypeWithTupleData.add(new CustomTypeWithTuple()); - customNestedTypeData.add(new NestedCustomType()); - } - - @Test - void testJoinKeyFields1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyFields2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, incompatible join key types - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(2)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyFields3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, incompatible number of join keys - assertThatThrownBy(() -> ds1.join(ds2).where(0, 1).equalTo(2)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyFields4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, join key out of range - assertThatThrownBy(() -> ds1.join(ds2).where(5).equalTo(0)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinKeyFields5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, negative key field position - assertThatThrownBy(() -> ds1.join(ds2).where(-1).equalTo(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinKeyFields6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, join key fields on custom type - assertThatThrownBy(() -> ds1.join(ds2).where(4).equalTo(0)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyExpressions1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.join(ds2).where("myInt").equalTo("myInt"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyExpressionsNested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customNestedTypeData); - DataSet ds2 = env.fromCollection(customNestedTypeData); - - // should work - try { - ds1.join(ds2).where("myInt").equalTo("myInt"); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyExpressions2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible join key types - assertThatThrownBy(() -> ds1.join(ds2).where("myInt").equalTo("myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyExpressions3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible number of join keys - assertThatThrownBy(() -> ds1.join(ds2).where("myInt", "myString").equalTo("myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyExpressions4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, join key non-existent - assertThatThrownBy(() -> ds1.join(ds2).where("myNonExistent").equalTo("myInt")) - .isInstanceOf(IllegalArgumentException.class); - } - - /** Test if mixed types of key selectors are properly working. */ - @Test - void testJoinKeyMixedKeySelector() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - try { - ds1.join(ds2) - .where("myInt") - .equalTo( - new KeySelector() { - @Override - public Integer getKey(CustomType value) throws Exception { - return value.myInt; - } - }); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixedKeySelectorTurned() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - try { - ds1.join(ds2) - .where( - new KeySelector() { - @Override - public Integer getKey(CustomType value) throws Exception { - return value.myInt; - } - }) - .equalTo("myInt"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixedTupleIndex() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - try { - ds1.join(ds2).where("f0").equalTo(4); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyNestedTuples() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet, Long, String, Long, Integer>> ds1 = - env.fromCollection(emptyNestedTupleData, nestedTupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - try { - ds1.join(ds2).where("f0.f0").equalTo(4); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyNestedTuplesWithCustom() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyNestedCustomTupleData, nestedCustomTupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - try { - TypeInformation t = ds1.join(ds2).where("f0.myInt").equalTo(4).getType(); - assertThat(t).as("not a composite type").isInstanceOf(CompositeType.class); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyWithCustomContainingTuple0() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeWithTupleData); - DataSet ds2 = env.fromCollection(customTypeWithTupleData); - try { - ds1.join(ds2).where("intByString.f0").equalTo("myInt"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyWithCustomContainingTuple1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeWithTupleData); - DataSet ds2 = env.fromCollection(customTypeWithTupleData); - try { - ds1.join(ds2).where("nested.myInt").equalTo("intByString.f0"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyWithCustomContainingTuple2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeWithTupleData); - DataSet ds2 = env.fromCollection(customTypeWithTupleData); - try { - ds1.join(ds2) - .where("nested.myInt", "myInt", "intByString.f1") - .equalTo("intByString.f0", "myInt", "myString"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyNestedTuplesWrongType() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet, Long, String, Long, Integer>> ds1 = - env.fromCollection(emptyNestedTupleData, nestedTupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - assertThatThrownBy( - () -> { - ds1.join(ds2).where("f0.f1").equalTo(4); // f0.f1 is a String - }) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyMixedTupleIndexTurned() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - try { - ds1.join(ds2).where(0).equalTo("f0"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixedTupleIndexWrongType() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - assertThatThrownBy( - () -> { - ds1.join(ds2) - .where("f0") - .equalTo(3); // 3 is of type long, so it should fail - }) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyMixedTupleIndex2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - try { - ds1.join(ds2).where("myInt").equalTo(4); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixedWrong() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - // wrongly mix String and Integer - assertThatThrownBy( - () -> - ds1.join(ds2) - .where("myString") - .equalTo( - (KeySelector) - value -> value.myInt)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyExpressions1Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.join(ds2).where("nested.myInt").equalTo("nested.myInt"); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyExpressions2Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible join key types - assertThatThrownBy(() -> ds1.join(ds2).where("nested.myInt").equalTo("nested.myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyExpressions3Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible number of join keys - assertThatThrownBy( - () -> - ds1.join(ds2) - .where("nested.myInt", "nested.myString") - .equalTo("nested.myString")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyExpressions4Nested() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, join key non-existent - assertThatThrownBy( - () -> ds1.join(ds2).where("nested.myNonExistent").equalTo("nested.myInt")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testJoinKeySelectors1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.join(ds2) - .where((KeySelector) value -> value.myLong) - .equalTo((KeySelector) value -> value.myLong); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixing1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where((KeySelector) value -> value.myLong).equalTo(3); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixing2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should work - try { - ds1.join(ds2).where(3).equalTo((KeySelector) value -> value.myLong); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinKeyMixing3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, incompatible types - assertThatThrownBy( - () -> - ds1.join(ds2) - .where(2) - .equalTo( - (KeySelector) - value -> value.myLong)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyMixing4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromCollection(customTypeData); - - // should not work, more than one key field position - assertThatThrownBy( - () -> - ds1.join(ds2) - .where(1, 3) - .equalTo( - (KeySelector) - value -> value.myLong)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyAtomic1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 0); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - ds1.join(ds2).where("*").equalTo(0); - } - - @Test - void testJoinKeyAtomic2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromElements(0, 0, 0); - - ds1.join(ds2).where(0).equalTo("*"); - } - - @Test - void testJoinKeyInvalidAtomic1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 0); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - assertThatThrownBy(() -> ds1.join(ds2).where("*", "invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyInvalidAtomic2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromElements(0, 0, 0); - - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo("*", "invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyInvalidAtomic3() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 0); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - assertThatThrownBy(() -> ds1.join(ds2).where("invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyInvalidAtomic4() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet ds2 = env.fromElements(0, 0, 0); - - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo("invalidKey")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyInvalidAtomic5() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = env.fromElements(new ArrayList()); - DataSet ds2 = env.fromElements(0, 0, 0); - - assertThatThrownBy(() -> ds1.join(ds2).where("*").equalTo("*")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinKeyInvalidAtomic6() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 0, 0); - DataSet> ds2 = env.fromElements(new ArrayList()); - - assertThatThrownBy(() -> ds1.join(ds2).where("*").equalTo("*")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testJoinProjection1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0).projectFirst(0); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection21() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0).projectFirst(0); - } catch (Exception e) { - fail(e.getMessage()); - } - - // should not work: field index is out of bounds of input tuple - try { - ds1.join(ds2).where(0).equalTo(0).projectFirst(-1); - fail(null); - } catch (IndexOutOfBoundsException iob) { - // we're good here - } catch (Exception e) { - fail(e.getMessage()); - } - - // should not work: field index is out of bounds of input tuple - try { - ds1.join(ds2).where(0).equalTo(0).project(9); - fail(null); - } catch (IndexOutOfBoundsException iob) { - // we're good here - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0).projectFirst(0, 3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0).projectFirst(0).projectSecond(3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2) - .where(0) - .equalTo(0) - .projectFirst(0, 2) - .projectSecond(1, 4) - .projectFirst(1); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection5() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2) - .where(0) - .equalTo(0) - .projectSecond(0, 2) - .projectFirst(1, 4) - .projectFirst(1); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection6() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - // should work - try { - ds1.join(ds2) - .where((KeySelector) value -> value.myLong) - .equalTo((KeySelector) value -> value.myLong) - .projectFirst() - .projectSecond(); - } catch (Exception e) { - System.out.println("FAILED: " + e); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection26() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromCollection(customTypeData); - DataSet ds2 = env.fromCollection(customTypeData); - // should work - try { - ds1.join(ds2) - .where((KeySelector) value -> value.myLong) - .equalTo((KeySelector) value -> value.myLong) - .projectFirst() - .projectSecond(); - } catch (Exception e) { - System.out.println("FAILED: " + e); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection7() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0).projectSecond().projectFirst(1, 4); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection27() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - ds1.join(ds2).where(0).equalTo(0).projectSecond().projectFirst(1, 4); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testJoinProjection8() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectFirst(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinProjection28() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectFirst(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinProjection9() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectSecond(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinProjection29() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectSecond(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - void testJoinProjection10() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.join(ds2).where(0).equalTo(0).projectFirst(2); - } - - @Test - void testJoinProjection30() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, type does not match - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectFirst(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - void testJoinProjection11() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, type does not match - ds1.join(ds2).where(0).equalTo(0).projectSecond(2); - } - - void testJoinProjection12() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.join(ds2).where(0).equalTo(0).projectSecond(2).projectFirst(1); - } - - @Test - void testJoinProjection13() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectSecond(0).projectFirst(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinProjection33() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy( - () -> ds1.join(ds2).where(0).equalTo(0).projectSecond(-1).projectFirst(3)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinProjection14() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy(() -> ds1.join(ds2).where(0).equalTo(0).projectFirst(0).projectSecond(5)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testJoinProjection34() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, index out of range - assertThatThrownBy( - () -> ds1.join(ds2).where(0).equalTo(0).projectFirst(0).projectSecond(-1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testSemanticPropsWithKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> tupleDs2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - JoinOperator joinOp = - tupleDs1.join(tupleDs2) - .where(new DummyTestKeySelector()) - .equalTo(new DummyTestKeySelector()) - .with(new DummyTestJoinFunction1()); - - SemanticProperties semProps = joinOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(1, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 6)).containsExactly(0); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 4, 6); - assertThat(semProps.getReadFields(1)).containsExactly(5, 3); - } - - @Test - void testSemanticPropsWithKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> tupleDs2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - JoinOperator joinOp = - tupleDs1.join(tupleDs2) - .where(new DummyTestKeySelector()) - .equalTo(new DummyTestKeySelector()) - .with(new DummyTestJoinFunction2()) - .withForwardedFieldsFirst("2;4->0") - .withForwardedFieldsSecond("0->4;1;1->3"); - - SemanticProperties semProps = joinOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(0); - - assertThat(semProps.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(1, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(1, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 6)).isEmpty(); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 3, 4); - - assertThat(semProps.getReadFields(1)).isNull(); - } - - @Test - void testSemanticPropsWithKeySelector3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> tupleDs2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - JoinOperator joinOp = - tupleDs1.join(tupleDs2) - .where(new DummyTestKeySelector()) - .equalTo(new DummyTestKeySelector()) - .projectFirst(2) - .projectSecond(0, 0, 3) - .projectFirst(0, 4) - .projectSecond(2); - - SemanticProperties semProps = joinOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(0); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(5); - - assertThat(semProps.getForwardingTargetFields(1, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 2)).containsExactly(1, 2); - assertThat(semProps.getForwardingTargetFields(1, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(1, 4)).containsExactly(6); - assertThat(semProps.getForwardingTargetFields(1, 5)).containsExactly(3); - assertThat(semProps.getForwardingTargetFields(1, 6)).isEmpty(); - } - - /* - * #################################################################### - */ - - /** Custom type for testing. */ - public static class Nested implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - - public Nested() {} - - public Nested(int i, long l, String s) { - myInt = i; - } - - @Override - public String toString() { - return "" + myInt; - } - } - - /** Simple nested type (only basic types). */ - public static class NestedCustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - public Nested nest; - - public NestedCustomType() {} - - public NestedCustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString + "," + nest; - } - } - - /** Custom type for testing. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public NestedCustomType nested; - public String myString; - public Object nothing; - public List countries; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - countries = null; - nested = new NestedCustomType(i, l, s); - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } - - /** Custom type for testing. */ - public static class CustomTypeWithTuple implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public NestedCustomType nested; - public String myString; - public Tuple2 intByString; - - public CustomTypeWithTuple() {} - - public CustomTypeWithTuple(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - nested = new NestedCustomType(i, l, s); - intByString = new Tuple2<>(i, s); - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } - - private static class DummyTestKeySelector - implements KeySelector< - Tuple5, Tuple2> { - @Override - public Tuple2 getKey(Tuple5 value) - throws Exception { - return new Tuple2<>(); - } - } - - @FunctionAnnotation.ForwardedFieldsFirst("0->4;1;1->3") - @FunctionAnnotation.ForwardedFieldsSecond("2;4->0") - @FunctionAnnotation.ReadFieldsFirst("0;2;4") - @FunctionAnnotation.ReadFieldsSecond("1;3") - private static class DummyTestJoinFunction1 - implements JoinFunction< - Tuple5, - Tuple5, - Tuple5> { - @Override - public Tuple5 join( - Tuple5 first, - Tuple5 second) - throws Exception { - return new Tuple5<>(); - } - } - - @FunctionAnnotation.ReadFieldsFirst("0;1;2") - private static class DummyTestJoinFunction2 - implements JoinFunction< - Tuple5, - Tuple5, - Tuple5> { - @Override - public Tuple5 join( - Tuple5 first, - Tuple5 second) - throws Exception { - return new Tuple5<>(); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java deleted file mode 100644 index e1b72c08f1232..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java +++ /dev/null @@ -1,272 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.CompositeType; -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.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DataSet#leftOuterJoin(DataSet)}. */ -class LeftOuterJoinOperatorTest { - - // TUPLE DATA - private static final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testLeftOuter1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2).where(0).equalTo(4).with(new DummyJoin()); - } - - @Test - void testLeftOuter2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2).where("f1").equalTo("f3").with(new DummyJoin()); - } - - @Test - void testLeftOuter3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2) - .where(new IntKeySelector()) - .equalTo(new IntKeySelector()) - .with(new DummyJoin()); - } - - @Test - void testLeftOuter4() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2).where(0).equalTo(new IntKeySelector()).with(new DummyJoin()); - } - - @Test - void testLeftOuter5() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2).where(new IntKeySelector()).equalTo("f4").with(new DummyJoin()); - } - - @Test - void testLeftOuter6() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2).where("f0").equalTo(4).with(new DummyJoin()); - } - - @Test - void testLeftOuter7() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // invalid key position - assertThatThrownBy(() -> ds1.leftOuterJoin(ds2).where(5).equalTo(0).with(new DummyJoin())) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testLeftOuter8() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // invalid key reference - assertThatThrownBy( - () -> ds1.leftOuterJoin(ds2).where(1).equalTo("f5").with(new DummyJoin())) - .isInstanceOf(CompositeType.InvalidFieldReferenceException.class); - } - - @Test - void testLeftOuter9() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // key types do not match - assertThatThrownBy(() -> ds1.leftOuterJoin(ds2).where(0).equalTo(1).with(new DummyJoin())) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testLeftOuter10() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // key types do not match - assertThatThrownBy( - () -> - ds1.leftOuterJoin(ds2) - .where(new IntKeySelector()) - .equalTo(new LongKeySelector()) - .with(new DummyJoin())) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testLeftOuterStrategy1() { - this.testLeftOuterStrategies(JoinHint.OPTIMIZER_CHOOSES); - } - - @Test - void testLeftOuterStrategy2() { - this.testLeftOuterStrategies(JoinHint.REPARTITION_SORT_MERGE); - } - - @Test - void testLeftOuterStrategy3() { - this.testLeftOuterStrategies(JoinHint.REPARTITION_HASH_SECOND); - } - - @Test - void testLeftOuterStrategy4() { - this.testLeftOuterStrategies(JoinHint.BROADCAST_HASH_SECOND); - } - - @Test - void testLeftOuterStrategy5() { - this.testLeftOuterStrategies(JoinHint.REPARTITION_HASH_FIRST); - } - - @Test - void testLeftOuterStrategy6() { - assertThatThrownBy(() -> this.testLeftOuterStrategies(JoinHint.BROADCAST_HASH_FIRST)) - .isInstanceOf(InvalidProgramException.class); - } - - private void testLeftOuterStrategies(JoinHint hint) { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.leftOuterJoin(ds2, hint).where(0).equalTo(4).with(new DummyJoin()); - } - - /* - * #################################################################### - */ - - @SuppressWarnings("serial") - private static class DummyJoin - implements JoinFunction< - Tuple5, - Tuple5, - Long> { - - @Override - public Long join( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return 1L; - } - } - - @SuppressWarnings("serial") - private static class IntKeySelector - implements KeySelector, Integer> { - - @Override - public Integer getKey(Tuple5 v) throws Exception { - return v.f0; - } - } - - @SuppressWarnings("serial") - private static class LongKeySelector - implements KeySelector, Long> { - - @Override - public Long getKey(Tuple5 v) throws Exception { - return v.f1; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java deleted file mode 100644 index 3ba3366847f56..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java +++ /dev/null @@ -1,260 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.operators.UnsortedGrouping; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.types.Row; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DataSet#maxBy(int...)}. */ -class MaxByOperatorTest { - - // TUPLE DATA - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - /** This test validates that no exceptions is thrown when an empty dataset calls maxBy(). */ - @Test - void testMaxByKeyFieldsDataset() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.maxBy(4, 0, 1, 2, 3); - } - - private final List customTypeData = new ArrayList<>(); - - /** - * This test validates that an InvalidProgramException is thrown when maxBy is used on a custom - * data type. - */ - @Test - void testCustomKeyFieldsDataset() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - // should not work: groups on custom type - assertThatThrownBy(() -> customDs.maxBy(0)).isInstanceOf(InvalidProgramException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsDataset1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.maxBy(5)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsDataset2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.maxBy(-1)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsDataset3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.maxBy(1, 2, 3, 4, -1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - // ---------------------------- GROUPING TESTS BELOW -------------------------------------- - - /** This test validates that no exceptions is thrown when an empty grouping calls maxBy(). */ - @Test - void testMaxByKeyFieldsGrouping() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should work - groupDs.maxBy(4, 0, 1, 2, 3); - } - - /** - * This test validates that an InvalidProgramException is thrown when maxBy is used on a custom - * data type. - */ - @Test - void testCustomKeyFieldsGrouping() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - // should not work: groups on custom type - assertThatThrownBy( - () -> { - UnsortedGrouping groupDs = - env.fromCollection(customTypeData).groupBy(0); - groupDs.maxBy(0); - }) - .isInstanceOf(InvalidProgramException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsGrouping1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> groupDs.maxBy(5)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsGrouping2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> groupDs.maxBy(-1)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsGrouping3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> groupDs.maxBy(1, 2, 3, 4, -1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - /** Validates that no ClassCastException happens should not fail e.g. like in FLINK-8255. */ - @Test - void testMaxByRowTypeInfoKeyFieldsDataset() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT}; - - String[] fieldNames = new String[] {"id", "value"}; - RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); - DataSet tupleDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo); - - assertThatThrownBy(() -> tupleDs.maxBy(0)).isInstanceOf(InvalidProgramException.class); - } - - /** Validates that no ClassCastException happens should not fail e.g. like in FLINK-8255. */ - @Test - void testMaxByRowTypeInfoKeyFieldsForUnsortedGrouping() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT}; - - String[] fieldNames = new String[] {"id", "value"}; - RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); - - UnsortedGrouping groupDs = - env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0); - - assertThatThrownBy(() -> groupDs.maxBy(1)).isInstanceOf(InvalidProgramException.class); - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java deleted file mode 100644 index fa9bdfc9cd925..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java +++ /dev/null @@ -1,269 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.operators.UnsortedGrouping; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.types.Row; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#minBy(int...)}. */ -class MinByOperatorTest { - - // TUPLE DATA - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - /** This test validates that no exceptions is thrown when an empty dataset calls minBy(). */ - @Test - void testMinByKeyFieldsDataset() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.minBy(4, 0, 1, 2, 3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - private final List customTypeData = new ArrayList(); - - /** - * This test validates that an InvalidProgramException is thrown when minBy is used on a custom - * data type. - */ - @Test - void testCustomKeyFieldsDataset() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - DataSet customDs = env.fromCollection(customTypeData); - // should not work: groups on custom type - assertThatThrownBy(() -> customDs.minBy(0)).isInstanceOf(InvalidProgramException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsDataset1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.minBy(5)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsDataset2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.minBy(-1)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsDataset3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> tupleDs.minBy(1, 2, 3, 4, -1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - // ---------------------------- GROUPING TESTS BELOW -------------------------------------- - - /** This test validates that no exceptions is thrown when an empty grouping calls minBy(). */ - @Test - void testMinByKeyFieldsGrouping() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should work - try { - groupDs.minBy(4, 0, 1, 2, 3); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - /** - * This test validates that an InvalidProgramException is thrown when minBy is used on a custom - * data type. - */ - @Test - void testCustomKeyFieldsGrouping() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - this.customTypeData.add(new CustomType()); - - // should not work: groups on custom type - assertThatThrownBy( - () -> { - UnsortedGrouping groupDs = - env.fromCollection(customTypeData).groupBy(0); - groupDs.minBy(0); - }) - .isInstanceOf(InvalidProgramException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsGrouping1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> groupDs.minBy(5)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsGrouping2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> groupDs.minBy(-1)).isInstanceOf(IndexOutOfBoundsException.class); - } - - /** - * This test validates that an index which is out of bounds throws an IndexOutOfBoundsException. - */ - @Test - void testOutOfTupleBoundsGrouping3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - UnsortedGrouping> groupDs = - env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0); - - // should not work, key out of tuple bounds - assertThatThrownBy(() -> groupDs.minBy(1, 2, 3, 4, -1)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - /** Validates that no ClassCastException happens should not fail e.g. like in FLINK-8255. */ - @Test - void testMinByRowTypeInfoKeyFieldsDataset() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT}; - - String[] fieldNames = new String[] {"id", "value"}; - RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); - DataSet tupleDs = env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo); - - assertThatThrownBy(() -> tupleDs.minBy(0)).isInstanceOf(InvalidProgramException.class); - } - - /** Validates that no ClassCastException happens should not fail e.g. like in FLINK-8255. */ - @Test - void testMinByRowTypeInfoKeyFieldsForUnsortedGrouping() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - TypeInformation[] types = new TypeInformation[] {Types.INT, Types.INT}; - - String[] fieldNames = new String[] {"id", "value"}; - RowTypeInfo rowTypeInfo = new RowTypeInfo(types, fieldNames); - - UnsortedGrouping groupDs = - env.fromCollection(Collections.singleton(new Row(2)), rowTypeInfo).groupBy(0); - - assertThatThrownBy(() -> groupDs.minBy(1)).isInstanceOf(InvalidProgramException.class); - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java deleted file mode 100644 index ba269caa7a234..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java +++ /dev/null @@ -1,78 +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.api.java.operator; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.operators.ResourceSpec; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.Operator; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.lang.reflect.Method; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link Operator}. */ -class OperatorTest { - - @Test - void testConfigurationOfParallelism() { - Operator operator = new MockOperator(); - - // verify explicit change in parallelism - int parallelism = 36; - operator.setParallelism(parallelism); - - assertThat(operator.getParallelism()).isEqualTo(parallelism); - - // verify that parallelism is reset to default flag value - parallelism = ExecutionConfig.PARALLELISM_DEFAULT; - operator.setParallelism(parallelism); - - assertThat(operator.getParallelism()).isEqualTo(parallelism); - } - - @Test - void testConfigurationOfResource() throws Exception { - Operator operator = new MockOperator(); - - Method opMethod = - Operator.class.getDeclaredMethod( - "setResources", ResourceSpec.class, ResourceSpec.class); - opMethod.setAccessible(true); - - // verify explicit change in resources - ResourceSpec minResources = ResourceSpec.newBuilder(1.0, 100).build(); - ResourceSpec preferredResources = ResourceSpec.newBuilder(2.0, 200).build(); - opMethod.invoke(operator, minResources, preferredResources); - - assertThat(operator.getMinResources()).isEqualTo(minResources); - assertThat(operator.getPreferredResources()).isEqualTo(preferredResources); - } - - private class MockOperator extends Operator { - public MockOperator() { - super( - ExecutionEnvironment.createCollectionsEnvironment(), - ValueTypeInfo.NULL_VALUE_TYPE_INFO); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java deleted file mode 100644 index e7b42d4315916..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java +++ /dev/null @@ -1,387 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -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.DataSource; -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.assertThatThrownBy; - -/** Tests for partitioning. */ -class PartitionOperatorTest { - - /** Custom data type, for testing purposes. */ - public static class CustomPojo implements Serializable, Comparable { - private Integer number; - private String name; - - public CustomPojo() {} - - public CustomPojo(Integer number, String name) { - this.number = number; - this.name = name; - } - - public Integer getNumber() { - return number; - } - - public void setNumber(Integer number) { - this.number = number; - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - @Override - public int compareTo(CustomPojo o) { - return Integer.compare(this.number, o.number); - } - } - - /** Custom data type with nested type, for testing purposes. */ - public static class NestedPojo implements Serializable { - private CustomPojo nested; - private Long outer; - - public NestedPojo() {} - - public NestedPojo(CustomPojo nested, Long outer) { - this.nested = nested; - this.outer = outer; - } - - public CustomPojo getNested() { - return nested; - } - - public void setNested(CustomPojo nested) { - this.nested = nested; - } - - public Long getOuter() { - return outer; - } - - public void setOuter(Long outer) { - this.outer = outer; - } - } - - private DataSet> getTupleDataSet(ExecutionEnvironment env) { - return env.fromElements( - new Tuple2<>(1, "first"), - new Tuple2<>(2, "second"), - new Tuple2<>(3, "third"), - new Tuple2<>(4, "fourth"), - new Tuple2<>(5, "fifth"), - new Tuple2<>(6, "sixth")); - } - - private DataSet getPojoDataSet(ExecutionEnvironment env) { - return env.fromElements( - new CustomPojo(1, "first"), - new CustomPojo(2, "second"), - new CustomPojo(3, "third"), - new CustomPojo(4, "fourth"), - new CustomPojo(5, "fifth"), - new CustomPojo(6, "sixth")); - } - - private DataSet getNestedPojoDataSet(ExecutionEnvironment env) { - return env.fromElements( - new NestedPojo(new CustomPojo(1, "first"), 1L), - new NestedPojo(new CustomPojo(2, "second"), 2L), - new NestedPojo(new CustomPojo(3, "third"), 3L), - new NestedPojo(new CustomPojo(4, "fourth"), 4L), - new NestedPojo(new CustomPojo(5, "fifth"), 5L), - new NestedPojo(new CustomPojo(6, "sixth"), 6L)); - } - - @Test - void testRebalance() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.rebalance(); - } - - @Test - void testHashPartitionByField1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.partitionByHash(0); - } - - @Test - void testHashPartitionByField2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.partitionByHash(0, 1); - } - - @Test - void testHashPartitionByFieldOutOfRange() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - assertThatThrownBy(() -> ds.partitionByHash(0, 1, 2)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testHashPartitionByFieldName1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - ds.partitionByHash("number"); - } - - @Test - void testHashPartitionByFieldName2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - ds.partitionByHash("number", "name"); - } - - @Test - void testHashPartitionByInvalidFieldName() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - assertThatThrownBy(() -> ds.partitionByHash("number", "name", "invalidField")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testRangePartitionByFieldName1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - ds.partitionByRange("number"); - } - - @Test - void testRangePartitionByFieldName2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - ds.partitionByRange("number", "name"); - } - - @Test - void testRangePartitionByInvalidFieldName() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - assertThatThrownBy(() -> ds.partitionByRange("number", "name", "invalidField")) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testRangePartitionByField1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.partitionByRange(0); - } - - @Test - void testRangePartitionByField2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.partitionByRange(0, 1); - } - - @Test - void testRangePartitionWithEmptyIndicesKey() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSource, Integer>> ds = - env.fromElements( - new Tuple2<>(new Tuple2<>(1, 1), 1), - new Tuple2<>(new Tuple2<>(2, 2), 2), - new Tuple2<>(new Tuple2<>(2, 2), 2)); - assertThatThrownBy(() -> ds.partitionByRange(new int[] {})) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testRangePartitionByFieldOutOfRange() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - assertThatThrownBy(() -> ds.partitionByRange(0, 1, 2)) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testHashPartitionWithOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - assertThatThrownBy(() -> ds.partitionByHash(1).withOrders(Order.ASCENDING)) - .isInstanceOf(IllegalStateException.class); - } - - @Test - void testRebalanceWithOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - assertThatThrownBy(() -> ds.rebalance().withOrders(Order.ASCENDING)) - .isInstanceOf(IllegalStateException.class); - } - - @Test - void testRangePartitionWithOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.partitionByRange(0).withOrders(Order.ASCENDING); - } - - @Test - void testRangePartitionWithTooManyOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - assertThatThrownBy( - () -> ds.partitionByRange(0).withOrders(Order.ASCENDING, Order.DESCENDING)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testRangePartitionByComplexKeyWithOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSource, Integer>> ds = - env.fromElements( - new Tuple2<>(new Tuple2<>(1, 1), 1), - new Tuple2<>(new Tuple2<>(2, 2), 2), - new Tuple2<>(new Tuple2<>(2, 2), 2)); - ds.partitionByRange(0, 1).withOrders(Order.ASCENDING, Order.DESCENDING); - } - - @Test - void testRangePartitionByComplexKeyWithTooManyOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSource, Integer>> ds = - env.fromElements( - new Tuple2<>(new Tuple2<>(1, 1), 1), - new Tuple2<>(new Tuple2<>(2, 2), 2), - new Tuple2<>(new Tuple2<>(2, 2), 2)); - assertThatThrownBy( - () -> ds.partitionByRange(0).withOrders(Order.ASCENDING, Order.DESCENDING)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testRangePartitionBySelectorComplexKeyWithOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getNestedPojoDataSet(env); - ds.partitionByRange((KeySelector) NestedPojo::getNested) - .withOrders(Order.ASCENDING); - } - - @Test - void testRangePartitionBySelectorComplexKeyWithTooManyOrders() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getNestedPojoDataSet(env); - assertThatThrownBy( - () -> - ds.partitionByRange( - (KeySelector) - value -> value.getNested()) - .withOrders(Order.ASCENDING, Order.DESCENDING)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testRangePartitionCustomPartitionerByFieldId() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - ds.partitionCustom((Partitioner) (key, numPartitions) -> 1, 0); - } - - @Test - void testRangePartitionInvalidCustomPartitionerByFieldId() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet> ds = getTupleDataSet(env); - assertThatThrownBy( - () -> - ds.partitionCustom( - (Partitioner) (key, numPartitions) -> 1, 1)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testRangePartitionCustomPartitionerByFieldName() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - ds.partitionCustom((Partitioner) (key, numPartitions) -> 1, "number"); - } - - @Test - void testRangePartitionInvalidCustomPartitionerByFieldName() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - assertThatThrownBy( - () -> - ds.partitionCustom( - (Partitioner) (key, numPartitions) -> 1, "name")) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testRangePartitionCustomPartitionerByKeySelector() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - final DataSet ds = getPojoDataSet(env); - ds.partitionCustom( - (Partitioner) (key, numPartitions) -> 1, - (KeySelector) CustomPojo::getNumber); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java deleted file mode 100644 index 69407d295f41f..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java +++ /dev/null @@ -1,114 +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.api.java.operator; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DataSet#project(int...)}. */ -class ProjectionOperatorTest { - - // TUPLE DATA - - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - // LONG DATA - - private final List emptyLongData = new ArrayList<>(); - - @Test - void testFieldsProjection() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.project(0); - - // should not work: too many fields - assertThatThrownBy( - () -> - tupleDs.project( - 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, - 17, 18, 19, 20, 21, 22, 23, 24, 25)) - .isInstanceOf(IllegalArgumentException.class); - - // should not work: index out of bounds of input tuple - assertThatThrownBy(() -> tupleDs.project(0, 5, 2)) - .isInstanceOf(IndexOutOfBoundsException.class); - - // should not work: not applied to tuple dataset - DataSet longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO); - assertThatThrownBy(() -> longDs.project(0)) - .isInstanceOf(UnsupportedOperationException.class); - } - - @Test - void testProjectionTypes() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.project(0); - - // should work: dummy types() here - tupleDs.project(2, 1, 4); - } - - @Test - void testProjectionWithoutTypes() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - tupleDs.project(2, 0, 4); - - // should not work: field index is out of bounds of input tuple - assertThatThrownBy(() -> tupleDs.project(2, -1, 4)) - .isInstanceOf(IndexOutOfBoundsException.class); - - // should not work: field index is out of bounds of input tuple - assertThatThrownBy(() -> tupleDs.project(2, 1, 4, 5, 8, 9)) - .isInstanceOf(IndexOutOfBoundsException.class); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java deleted file mode 100644 index 547f236dcb56b..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java +++ /dev/null @@ -1,231 +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.api.java.operator; - -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.ReduceOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link DataSet#reduce(ReduceFunction)}. */ -class ReduceOperatorTest { - - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testSemanticPropsWithKeySelector1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - ReduceOperator> reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()).reduce(new DummyReduceFunction1()); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(2); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 5, 6); - } - - @Test - void testSemanticPropsWithKeySelector2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - ReduceOperator> reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .reduce(new DummyReduceFunction2()) - .withForwardedFields("0->4;1;1->3;2"); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(4); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 4)).isEqualTo(2); - - assertThat(semProps.getReadFields(0)).containsExactly(2, 5, 6); - } - - @Test - void testSemanticPropsWithKeySelector3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - ReduceOperator> reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()) - .reduce(new DummyReduceFunction3()) - .withForwardedFields("4->0;3;3->1;2"); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 3)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 4)).containsExactly(2); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(1, 3); - assertThat(semProps.getForwardingTargetFields(0, 6)).containsExactly(0); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(6); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 2)).isEqualTo(4); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - @Test - void testSemanticPropsWithKeySelector4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - ReduceOperator> reduceOp = - tupleDs.groupBy(new DummyTestKeySelector()).reduce(new DummyReduceFunction4()); - - SemanticProperties semProps = reduceOp.getSemanticProperties(); - - assertThat(semProps.getForwardingTargetFields(0, 0)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 1)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 2)).containsExactly(0); - assertThat(semProps.getForwardingTargetFields(0, 3)).containsExactly(1); - assertThat(semProps.getForwardingTargetFields(0, 4)).isEmpty(); - assertThat(semProps.getForwardingTargetFields(0, 5)).containsExactly(3); - assertThat(semProps.getForwardingTargetFields(0, 6)).isEmpty(); - - assertThat(semProps.getForwardingSourceField(0, 0)).isEqualTo(2); - assertThat(semProps.getForwardingSourceField(0, 1)).isEqualTo(3); - assertThat(semProps.getForwardingSourceField(0, 2)).isLessThan(0); - assertThat(semProps.getForwardingSourceField(0, 3)).isEqualTo(5); - assertThat(semProps.getForwardingSourceField(0, 4)).isLessThan(0); - - assertThat(semProps.getReadFields(0)).isNull(); - } - - private static class DummyTestKeySelector - implements KeySelector< - Tuple5, Tuple2> { - @Override - public Tuple2 getKey(Tuple5 value) - throws Exception { - return new Tuple2<>(); - } - } - - @FunctionAnnotation.ForwardedFields("0->4;1;1->3;2") - @FunctionAnnotation.ReadFields("0;3;4") - private static class DummyReduceFunction1 - implements ReduceFunction> { - @Override - public Tuple5 reduce( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return new Tuple5<>(); - } - } - - @FunctionAnnotation.ReadFields("0;3;4") - private static class DummyReduceFunction2 - implements ReduceFunction> { - @Override - public Tuple5 reduce( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return new Tuple5<>(); - } - } - - private static class DummyReduceFunction3 - implements ReduceFunction> { - @Override - public Tuple5 reduce( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return new Tuple5<>(); - } - } - - @FunctionAnnotation.NonForwardedFields("2;4") - private static class DummyReduceFunction4 - implements ReduceFunction> { - @Override - public Tuple5 reduce( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return new Tuple5<>(); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java deleted file mode 100644 index 5b8f140d0218f..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java +++ /dev/null @@ -1,272 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.CompositeType; -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.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Tests for {@link DataSet#rightOuterJoin(DataSet)}. */ -class RightOuterJoinOperatorTest { - - // TUPLE DATA - private static final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - void testRightOuter1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2).where(0).equalTo(4).with(new DummyJoin()); - } - - @Test - void testRightOuter2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2).where("f1").equalTo("f3").with(new DummyJoin()); - } - - @Test - void testRightOuter3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2) - .where(new IntKeySelector()) - .equalTo(new IntKeySelector()) - .with(new DummyJoin()); - } - - @Test - void testRightOuter4() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2).where(0).equalTo(new IntKeySelector()).with(new DummyJoin()); - } - - @Test - void testRightOuter5() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2).where(new IntKeySelector()).equalTo("f4").with(new DummyJoin()); - } - - @Test - void testRightOuter6() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2).where("f0").equalTo(4).with(new DummyJoin()); - } - - @Test - void testRightOuter7() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // invalid key position - assertThatThrownBy(() -> ds1.rightOuterJoin(ds2).where(5).equalTo(0).with(new DummyJoin())) - .isInstanceOf(IndexOutOfBoundsException.class); - } - - @Test - void testRightOuter8() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // invalid key reference - assertThatThrownBy( - () -> ds1.rightOuterJoin(ds2).where(1).equalTo("f5").with(new DummyJoin())) - .isInstanceOf(CompositeType.InvalidFieldReferenceException.class); - } - - @Test - void testRightOuter9() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // key types do not match - assertThatThrownBy(() -> ds1.rightOuterJoin(ds2).where(0).equalTo(1).with(new DummyJoin())) - .isInstanceOf((InvalidProgramException.class)); - } - - @Test - void testRightOuter10() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // key types do not match - assertThatThrownBy( - () -> - ds1.rightOuterJoin(ds2) - .where(new IntKeySelector()) - .equalTo(new LongKeySelector()) - .with(new DummyJoin())) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testRightOuterStrategy1() { - this.testRightOuterStrategies(JoinHint.OPTIMIZER_CHOOSES); - } - - @Test - void testRightOuterStrategy2() { - this.testRightOuterStrategies(JoinHint.REPARTITION_SORT_MERGE); - } - - @Test - void testRightOuterStrategy3() { - this.testRightOuterStrategies(JoinHint.REPARTITION_HASH_SECOND); - } - - @Test - void testRightOuterStrategy4() { - assertThatThrownBy(() -> this.testRightOuterStrategies(JoinHint.BROADCAST_HASH_SECOND)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testRightOuterStrategy5() { - this.testRightOuterStrategies(JoinHint.REPARTITION_HASH_FIRST); - } - - @Test - public void testRightOuterStrategy6() { - this.testRightOuterStrategies(JoinHint.BROADCAST_HASH_FIRST); - } - - private void testRightOuterStrategies(JoinHint hint) { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - DataSet> ds2 = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - ds1.rightOuterJoin(ds2, hint).where(0).equalTo(4).with(new DummyJoin()); - } - - /* - * #################################################################### - */ - - @SuppressWarnings("serial") - private static class DummyJoin - implements JoinFunction< - Tuple5, - Tuple5, - Long> { - - @Override - public Long join( - Tuple5 v1, - Tuple5 v2) - throws Exception { - return 1L; - } - } - - @SuppressWarnings("serial") - private static class IntKeySelector - implements KeySelector, Integer> { - - @Override - public Integer getKey(Tuple5 v) throws Exception { - return v.f0; - } - } - - @SuppressWarnings("serial") - private static class LongKeySelector - implements KeySelector, Long> { - - @Override - public Long getKey(Tuple5 v) throws Exception { - return v.f1; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java deleted file mode 100644 index a187f5ab6acf6..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java +++ /dev/null @@ -1,314 +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.api.java.operator; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -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.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for {@link DataSet#sortPartition(int, Order)}. */ -class SortPartitionTest { - - // TUPLE DATA - private final List> emptyTupleData = - new ArrayList<>(); - - private final TupleTypeInfo> tupleTypeInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - private final TupleTypeInfo> tupleWithCustomInfo = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - TypeExtractor.createTypeInfo(CustomType.class), - BasicArrayTypeInfo.LONG_ARRAY_TYPE_INFO); - - // LONG DATA - private final List emptyLongData = new ArrayList<>(); - - private final List customTypeData = new ArrayList<>(); - - private final List> tupleWithCustomData = - new ArrayList<>(); - - @Test - void testSortPartitionPositionKeys1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.sortPartition(0, Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testSortPartitionPositionKeys2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.sortPartition(0, Order.ASCENDING).sortPartition(3, Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testSortPartitionWithPositionKeys3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy(() -> tupleDs.sortPartition(2, Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSortPartitionWithPositionKeys4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy(() -> tupleDs.sortPartition(3, Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSortPartitionExpressionKeys1() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(emptyTupleData, tupleTypeInfo); - - // should work - try { - tupleDs.sortPartition("f1", Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testSortPartitionExpressionKeys2() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should work - try { - tupleDs.sortPartition("f0", Order.ASCENDING) - .sortPartition("f2.nested.myInt", Order.DESCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testSortPartitionWithExpressionKeys3() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy(() -> tupleDs.sortPartition("f2.nested", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSortPartitionWithExpressionKeys4() { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy(() -> tupleDs.sortPartition("f3", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSortPartitionWithKeySelector1() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should work - try { - tupleDs.sortPartition( - (KeySelector, Integer>) - value -> value.f0, - Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testSortPartitionWithKeySelector2() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy( - () -> - tupleDs.sortPartition( - (KeySelector< - Tuple4, - Long[]>) - value -> value.f3, - Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSortPartitionWithKeySelector3() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy( - () -> - tupleDs.sortPartition("f1", Order.ASCENDING) - .sortPartition( - (KeySelector< - Tuple4< - Integer, - Long, - CustomType, - Long[]>, - CustomType>) - value -> value.f2, - Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - @Test - void testSortPartitionWithKeySelector4() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // should work - try { - tupleDs.sortPartition( - new KeySelector< - Tuple4, Tuple2>() { - @Override - public Tuple2 getKey( - Tuple4 value) throws Exception { - return new Tuple2<>(value.f0, value.f1); - } - }, - Order.ASCENDING); - } catch (Exception e) { - fail(e.getMessage()); - } - } - - @Test - void testSortPartitionWithKeySelector5() { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> tupleDs = - env.fromCollection(tupleWithCustomData, tupleWithCustomInfo); - - // must not work - assertThatThrownBy( - () -> - tupleDs.sortPartition( - (KeySelector< - Tuple4< - Integer, - Long, - CustomType, - Long[]>, - CustomType>) - value -> value.f2, - Order.ASCENDING) - .sortPartition("f1", Order.ASCENDING)) - .isInstanceOf(InvalidProgramException.class); - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - /** Custom nested data type, for testing purposes. */ - public static class Nest { - public int myInt; - } - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - public Nest nested; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java deleted file mode 100644 index 72cb8d1f70a55..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java +++ /dev/null @@ -1,150 +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.api.java.operators; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -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.operators.translation.PlanFilterOperator; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.util.Collector; -import org.apache.flink.util.Visitor; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Test proper automated assignment of the transformation's name, if not set by the user. */ -@SuppressWarnings("serial") -class NamesTest implements Serializable { - - @Test - void testDefaultName() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet strs = env.fromCollection(Arrays.asList("a", "b")); - - // WARNING: The test will fail if this line is being moved down in the file (the line-number - // is hard-coded) - strs.filter( - new FilterFunction() { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(String value) throws Exception { - return value.equals("a"); - } - }) - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - testForName("Filter at testDefaultName(NamesTest.java:55)", plan); - } - - @Test - void testGivenName() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet strs = env.fromCollection(Arrays.asList("a", "b")); - strs.filter( - new FilterFunction() { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(String value) throws Exception { - return value.equals("a"); - } - }) - .name("GivenName") - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - testForName("GivenName", plan); - } - - @Test - void testJoinWith() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - List> strLi = new ArrayList<>(); - strLi.add(new Tuple1<>("a")); - strLi.add(new Tuple1<>("b")); - - DataSet> strs = env.fromCollection(strLi); - DataSet> strs1 = env.fromCollection(strLi); - strs.join(strs1) - .where(0) - .equalTo(0) - .with( - new FlatJoinFunction, Tuple1, String>() { - @Override - public void join( - Tuple1 first, - Tuple1 second, - Collector out) - throws Exception { - // - } - }) - .output(new DiscardingOutputFormat<>()); - Plan plan = env.createProgramPlan(); - plan.accept( - new Visitor>() { - @Override - public boolean preVisit(Operator visitable) { - if (visitable instanceof InnerJoinOperatorBase) { - assertThat(visitable.getName()) - .isEqualTo("Join at testJoinWith(NamesTest.java:102)"); - } - return true; - } - - @Override - public void postVisit(Operator visitable) {} - }); - } - - private static void testForName(final String expected, Plan plan) { - plan.accept( - new Visitor>() { - @Override - public boolean preVisit(Operator visitable) { - if (visitable instanceof PlanFilterOperator) { - // cast is actually not required. Its just a check for the right element - PlanFilterOperator filterOp = (PlanFilterOperator) visitable; - assertThat(filterOp.getName()).isEqualTo(expected); - } - return true; - } - - @Override - public void postVisit(Operator visitable) { - // - } - }); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java deleted file mode 100644 index ec05e42498062..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/AggregateTranslationTest.java +++ /dev/null @@ -1,76 +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.api.java.operators.translation; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.types.StringValue; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for translation of aggregations. */ -class AggregateTranslationTest { - - @Test - void translateAggregate() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - @SuppressWarnings("unchecked") - DataSet> initialData = - env.fromElements(new Tuple3<>(3.141592, new StringValue("foobar"), 77L)); - - initialData - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .and(Aggregations.SUM, 2) - .output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - GroupReduceOperatorBase reducer = - (GroupReduceOperatorBase) sink.getInput(); - - // check keys - assertThat(reducer.getKeyColumns(0)).containsExactly(0); - - assertThat(reducer.getParallelism()).isEqualTo(-1); - assertThat(reducer.isCombinable()).isTrue(); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java deleted file mode 100644 index 23c9c7ff8be3e..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java +++ /dev/null @@ -1,96 +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.api.java.operators.translation; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.operators.base.BulkIterationBase; -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.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple3; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -/** - * Simple test for validating the parallelism of a bulk iteration. This test is not as comprehensive - * as {@link DeltaIterationTranslationTest}. - */ -@SuppressWarnings("serial") -class BulkIterationTranslationTest implements java.io.Serializable { - - @Test - void testCorrectTranslation() { - final String jobName = "Test JobName"; - - final int numIterations = 13; - - final int defaultParallelism = 133; - final int iterationParallelism = 77; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // ------------ construct the test program ------------------ - - { - env.setParallelism(defaultParallelism); - - @SuppressWarnings("unchecked") - DataSet> initialDataSet = - env.fromElements(new Tuple3<>(3.44, 5L, "abc")); - - IterativeDataSet> bulkIteration = - initialDataSet.iterate(numIterations); - bulkIteration.setParallelism(iterationParallelism); - - // test that multiple iteration consumers are supported - DataSet> identity = - bulkIteration.map(new IdentityMapper<>()); - - DataSet> result = bulkIteration.closeWith(identity); - - result.output(new DiscardingOutputFormat<>()); - result.writeAsText("/dev/null"); - } - - Plan p = env.createProgramPlan(jobName); - - // ------------- validate the plan ---------------- - - BulkIterationBase iteration = - (BulkIterationBase) p.getDataSinks().iterator().next().getInput(); - - assertThat(p.getJobName()).isEqualTo(jobName); - assertThat(p.getDefaultParallelism()).isEqualTo(defaultParallelism); - assertThat(iteration.getParallelism()).isEqualTo(iterationParallelism); - } - - // -------------------------------------------------------------------------------------------- - - private static class IdentityMapper extends RichMapFunction { - @Override - public T map(T value) throws Exception { - return value; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/CoGroupSortTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/CoGroupSortTranslationTest.java deleted file mode 100644 index 7d9a808ee5081..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/CoGroupSortTranslationTest.java +++ /dev/null @@ -1,148 +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.api.java.operators.translation; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; -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.java.tuple.Tuple3; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for translation of co-group sort. */ -@SuppressWarnings({"serial", "unchecked"}) -class CoGroupSortTranslationTest implements java.io.Serializable { - - @Test - void testGroupSortTuples() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> input2 = env.fromElements(new Tuple3<>(0L, 0L, 0L)); - - input1.coGroup(input2) - .where(1) - .equalTo(2) - .sortFirstGroup(0, Order.DESCENDING) - .sortSecondGroup(1, Order.ASCENDING) - .sortSecondGroup(0, Order.DESCENDING) - .with( - new CoGroupFunction< - Tuple2, Tuple3, Long>() { - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector out) - throws Exception {} - }) - .output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - CoGroupOperatorBase coGroup = - (CoGroupOperatorBase) sink.getInput(); - - assertThat(coGroup.getGroupOrderForInputOne()).isNotNull(); - assertThat(coGroup.getGroupOrderForInputTwo()).isNotNull(); - - assertThat(coGroup.getGroupOrderForInputOne().getNumberOfFields()).isOne(); - assertThat(coGroup.getGroupOrderForInputOne().getFieldNumber(0).intValue()).isZero(); - assertThat(coGroup.getGroupOrderForInputOne().getOrder(0)).isEqualTo(Order.DESCENDING); - - assertThat(coGroup.getGroupOrderForInputTwo().getNumberOfFields()).isEqualTo(2); - assertThat(coGroup.getGroupOrderForInputTwo().getFieldNumber(0).intValue()).isOne(); - assertThat(coGroup.getGroupOrderForInputTwo().getFieldNumber(1).intValue()).isZero(); - assertThat(coGroup.getGroupOrderForInputTwo().getOrder(0)).isEqualTo(Order.ASCENDING); - assertThat(coGroup.getGroupOrderForInputTwo().getOrder(1)).isEqualTo(Order.DESCENDING); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testSortTuplesAndPojos() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet input2 = env.fromElements(new TestPoJo()); - - input1.coGroup(input2) - .where(1) - .equalTo("b") - .sortFirstGroup(0, Order.DESCENDING) - .sortSecondGroup("c", Order.ASCENDING) - .sortSecondGroup("a", Order.DESCENDING) - .with( - new CoGroupFunction, TestPoJo, Long>() { - @Override - public void coGroup( - Iterable> first, - Iterable second, - Collector out) - throws Exception {} - }) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - CoGroupOperatorBase coGroup = - (CoGroupOperatorBase) sink.getInput(); - - assertThat(coGroup.getGroupOrderForInputOne()).isNotNull(); - assertThat(coGroup.getGroupOrderForInputTwo()).isNotNull(); - - assertThat(coGroup.getGroupOrderForInputOne().getNumberOfFields()).isOne(); - assertThat(coGroup.getGroupOrderForInputOne().getFieldNumber(0).intValue()).isZero(); - assertThat(coGroup.getGroupOrderForInputOne().getOrder(0)).isEqualTo(Order.DESCENDING); - - assertThat(coGroup.getGroupOrderForInputTwo().getNumberOfFields()).isEqualTo(2); - assertThat(coGroup.getGroupOrderForInputTwo().getFieldNumber(0).intValue()) - .isEqualTo(2); - assertThat(coGroup.getGroupOrderForInputTwo().getFieldNumber(1).intValue()).isZero(); - assertThat(coGroup.getGroupOrderForInputTwo().getOrder(0)).isEqualTo(Order.ASCENDING); - assertThat(coGroup.getGroupOrderForInputTwo().getOrder(1)).isEqualTo(Order.DESCENDING); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** Sample test pojo. */ - public static class TestPoJo { - public long a; - public long b; - public long c; - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java deleted file mode 100644 index cfd9f281aca6b..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DeltaIterationTranslationTest.java +++ /dev/null @@ -1,316 +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.api.java.operators.translation; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.common.functions.RichCoGroupFunction; -import org.apache.flink.api.common.functions.RichJoinFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.base.DeltaIterationBase; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.util.Collector; - -import org.junit.jupiter.api.Test; - -import java.util.Iterator; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for translation of delta iterations. */ -@SuppressWarnings("serial") -class DeltaIterationTranslationTest implements java.io.Serializable { - - @Test - void testCorrectTranslation() { - try { - final String jobName = "Test JobName"; - final String iterationName = "Test Name"; - - final String beforeNextWorksetMap = "Some Mapper"; - - final String aggregatorName = "AggregatorName"; - - final int[] iterationKeys = new int[] {2}; - final int numIterations = 13; - - final int defaultParallelism = 133; - final int iterationParallelism = 77; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // ------------ construct the test program ------------------ - { - env.setParallelism(defaultParallelism); - - @SuppressWarnings("unchecked") - DataSet> initialSolutionSet = - env.fromElements(new Tuple3<>(3.44, 5L, "abc")); - - @SuppressWarnings("unchecked") - DataSet> initialWorkSet = - env.fromElements(new Tuple2<>(1.23, "abc")); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta( - initialWorkSet, numIterations, iterationKeys); - iteration.name(iterationName).parallelism(iterationParallelism); - - iteration.registerAggregator(aggregatorName, new LongSumAggregator()); - - // test that multiple workset consumers are supported - DataSet> worksetSelfJoin = - iteration - .getWorkset() - .map(new IdentityMapper>()) - .join(iteration.getWorkset()) - .where(1) - .equalTo(1) - .projectFirst(0, 1); - - DataSet> joined = - worksetSelfJoin - .join(iteration.getSolutionSet()) - .where(1) - .equalTo(2) - .with(new SolutionWorksetJoin()); - - DataSet> result = - iteration.closeWith( - joined, - joined.map(new NextWorksetMapper()).name(beforeNextWorksetMap)); - - result.output(new DiscardingOutputFormat<>()); - result.writeAsText("/dev/null"); - } - - Plan p = env.createProgramPlan(jobName); - - // ------------- validate the plan ---------------- - assertThat(p.getJobName()).isEqualTo(jobName); - assertThat(p.getDefaultParallelism()).isEqualTo(defaultParallelism); - - // validate the iteration - GenericDataSinkBase sink1, sink2; - { - Iterator> sinks = p.getDataSinks().iterator(); - sink1 = sinks.next(); - sink2 = sinks.next(); - } - - DeltaIterationBase iteration = (DeltaIterationBase) sink1.getInput(); - - // check that multi consumer translation works for iterations - assertThat(sink2.getInput()).isEqualTo(iteration); - - // check the basic iteration properties - assertThat(iteration.getMaximumNumberOfIterations()).isEqualTo(numIterations); - assertThat(iteration.getSolutionSetKeyFields()).containsExactly(iterationKeys); - assertThat(iteration.getParallelism()).isEqualTo(iterationParallelism); - assertThat(iteration.getName()).isEqualTo(iterationName); - - MapOperatorBase nextWorksetMapper = - (MapOperatorBase) iteration.getNextWorkset(); - InnerJoinOperatorBase solutionSetJoin = - (InnerJoinOperatorBase) iteration.getSolutionSetDelta(); - InnerJoinOperatorBase worksetSelfJoin = - (InnerJoinOperatorBase) solutionSetJoin.getFirstInput(); - MapOperatorBase worksetMapper = - (MapOperatorBase) worksetSelfJoin.getFirstInput(); - - assertThat(worksetMapper.getUserCodeWrapper().getUserCodeClass()) - .isEqualTo(IdentityMapper.class); - assertThat(nextWorksetMapper.getUserCodeWrapper().getUserCodeClass()) - .isEqualTo(NextWorksetMapper.class); - if (solutionSetJoin.getUserCodeWrapper().getUserCodeObject() - instanceof WrappingFunction) { - WrappingFunction wf = - (WrappingFunction) - solutionSetJoin.getUserCodeWrapper().getUserCodeObject(); - assertThat(wf.getWrappedFunction().getClass()).isEqualTo(SolutionWorksetJoin.class); - } else { - assertThat(solutionSetJoin.getUserCodeWrapper().getUserCodeClass()) - .isEqualTo(SolutionWorksetJoin.class); - } - - assertThat(nextWorksetMapper.getName()).isEqualTo(beforeNextWorksetMap); - - assertThat( - iteration - .getAggregators() - .getAllRegisteredAggregators() - .iterator() - .next() - .getName()) - .isEqualTo(aggregatorName); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testRejectWhenSolutionSetKeysDontMatchJoin() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> initialSolutionSet = - env.fromElements(new Tuple3(3.44, 5L, "abc")); - - @SuppressWarnings("unchecked") - DataSet> initialWorkSet = - env.fromElements(new Tuple2(1.23, "abc")); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialWorkSet, 10, 1); - - try { - iteration.getWorkset().join(iteration.getSolutionSet()).where(1).equalTo(2); - fail("Accepted invalid program."); - } catch (InvalidProgramException e) { - // all good! - } - - try { - iteration.getSolutionSet().join(iteration.getWorkset()).where(2).equalTo(1); - fail("Accepted invalid program."); - } catch (InvalidProgramException e) { - // all good! - } - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - void testRejectWhenSolutionSetKeysDontMatchCoGroup() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> initialSolutionSet = - env.fromElements(new Tuple3(3.44, 5L, "abc")); - - @SuppressWarnings("unchecked") - DataSet> initialWorkSet = - env.fromElements(new Tuple2(1.23, "abc")); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialWorkSet, 10, 1); - - try { - iteration - .getWorkset() - .coGroup(iteration.getSolutionSet()) - .where(1) - .equalTo(2) - .with(new SolutionWorksetCoGroup1()); - fail("Accepted invalid program."); - } catch (InvalidProgramException e) { - // all good! - } - - try { - iteration - .getSolutionSet() - .coGroup(iteration.getWorkset()) - .where(2) - .equalTo(1) - .with(new SolutionWorksetCoGroup2()); - fail("Accepted invalid program."); - } catch (InvalidProgramException e) { - // all good! - } - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static class SolutionWorksetJoin - extends RichJoinFunction< - Tuple2, - Tuple3, - Tuple3> { - @Override - public Tuple3 join( - Tuple2 first, Tuple3 second) { - return null; - } - } - - private static class NextWorksetMapper - extends RichMapFunction, Tuple2> { - @Override - public Tuple2 map(Tuple3 value) { - return null; - } - } - - private static class IdentityMapper extends RichMapFunction { - - @Override - public T map(T value) throws Exception { - return value; - } - } - - private static class SolutionWorksetCoGroup1 - extends RichCoGroupFunction< - Tuple2, - Tuple3, - Tuple3> { - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) {} - } - - private static class SolutionWorksetCoGroup2 - extends RichCoGroupFunction< - Tuple3, - Tuple2, - Tuple3> { - - @Override - public void coGroup( - Iterable> second, - Iterable> first, - Collector> out) {} - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java deleted file mode 100644 index 78d14ae80d117..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java +++ /dev/null @@ -1,278 +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.api.java.operators.translation; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.junit.jupiter.api.Test; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for translation of distinct operation. */ -@SuppressWarnings("serial") -class DistinctTranslationTest { - - @Test - void translateDistinctPlain() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> initialData = getSourceDataSet(env); - - initialData.distinct().output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - // currently distinct is translated to a Reduce - ReduceOperatorBase reducer = (ReduceOperatorBase) sink.getInput(); - - // check types - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(initialData.getType()); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(initialData.getType()); - - // check keys - assertThat(reducer.getKeyColumns(0)).containsExactly(0, 1, 2); - - // parallelism was not configured on the operator - assertThat(reducer.getParallelism()).isIn(-1, 1); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateDistinctPlain2() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet initialData = getSourcePojoDataSet(env); - - initialData.distinct().output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - // currently distinct is translated to a Reduce - ReduceOperatorBase reducer = (ReduceOperatorBase) sink.getInput(); - - // check types - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(initialData.getType()); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(initialData.getType()); - - // check keys - assertThat(reducer.getKeyColumns(0)).containsExactly(0); - - // parallelism was not configured on the operator - assertThat(reducer.getParallelism()).isIn(-1, 1); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateDistinctPosition() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> initialData = getSourceDataSet(env); - - initialData.distinct(1, 2).output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - // currently distinct is translated to a Reduce - ReduceOperatorBase reducer = (ReduceOperatorBase) sink.getInput(); - - // check types - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(initialData.getType()); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(initialData.getType()); - - // check keys - assertThat(reducer.getKeyColumns(0)).containsExactly(1, 2); - - // parallelism was not configured on the operator - assertThat(reducer.getParallelism()).isIn(-1, 1); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateDistinctKeySelector() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> initialData = getSourceDataSet(env); - - initialData - .distinct( - (KeySelector, StringValue>) - value -> value.f1) - .setParallelism(4) - .output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - MapOperatorBase keyRemover = (MapOperatorBase) sink.getInput(); - PlanUnwrappingReduceOperator reducer = - (PlanUnwrappingReduceOperator) keyRemover.getInput(); - MapOperatorBase keyExtractor = (MapOperatorBase) reducer.getInput(); - - // check the parallelisms - assertThat(keyExtractor.getParallelism()).isOne(); - assertThat(reducer.getParallelism()).isEqualTo(4); - - // check types - TypeInformation keyValueInfo = - new TupleTypeInfo>>( - new ValueTypeInfo<>(StringValue.class), initialData.getType()); - - assertThat(keyExtractor.getOperatorInfo().getInputType()) - .isEqualTo(initialData.getType()); - assertThat(keyExtractor.getOperatorInfo().getOutputType()).isEqualTo(keyValueInfo); - - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(keyValueInfo); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(keyValueInfo); - - assertThat(keyRemover.getOperatorInfo().getInputType()).isEqualTo(keyValueInfo); - assertThat(keyRemover.getOperatorInfo().getOutputType()) - .isEqualTo(initialData.getType()); - - // check keys - assertThat(keyExtractor.getUserCodeWrapper().getUserCodeClass()) - .isEqualTo(KeyExtractingMapper.class); - - assertThat(keyExtractor.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateDistinctExpressionKey() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet initialData = getSourcePojoDataSet(env); - - initialData.distinct("myInt").output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - // currently distinct is translated to a Reduce - ReduceOperatorBase reducer = (ReduceOperatorBase) sink.getInput(); - - // check types - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(initialData.getType()); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(initialData.getType()); - - // check keys - assertThat(reducer.getKeyColumns(0)).containsExactly(0); - - // parallelism was not configured on the operator - assertThat(reducer.getParallelism()).isIn(-1, 1); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @SuppressWarnings("unchecked") - private static DataSet> getSourceDataSet( - ExecutionEnvironment env) { - return env.fromElements( - new Tuple3<>(3.141592, new StringValue("foobar"), new LongValue(77))) - .setParallelism(1); - } - - private static DataSet getSourcePojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new CustomType(1)); - return env.fromCollection(data); - } - - /** Custom data type, for testing purposes. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - public int myInt; - - public CustomType() {} - - public CustomType(int i) { - myInt = i; - } - - @Override - public String toString() { - return "" + myInt; - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java deleted file mode 100644 index 8d70203f9fe97..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/ReduceTranslationTests.java +++ /dev/null @@ -1,219 +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.api.java.operators.translation; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichReduceFunction; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for translation of reduce operation. */ -@SuppressWarnings("serial") -class ReduceTranslationTests implements java.io.Serializable { - - @Test - void translateNonGroupedReduce() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> initialData = getSourceDataSet(env); - - initialData - .reduce( - new RichReduceFunction>() { - public Tuple3 reduce( - Tuple3 value1, - Tuple3 value2) { - return value1; - } - }) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - ReduceOperatorBase reducer = (ReduceOperatorBase) sink.getInput(); - - // check types - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(initialData.getType()); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(initialData.getType()); - - // check keys - assertThat(reducer.getKeyColumns(0) == null || reducer.getKeyColumns(0).length == 0) - .isTrue(); - - // parallelism was not configured on the operator - assertThat( - reducer.getParallelism() == 1 - || reducer.getParallelism() - == ExecutionConfig.PARALLELISM_DEFAULT) - .isTrue(); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateGroupedReduceNoMapper() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> initialData = getSourceDataSet(env); - - initialData - .groupBy(2) - .reduce( - new RichReduceFunction>() { - public Tuple3 reduce( - Tuple3 value1, - Tuple3 value2) { - return value1; - } - }) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - ReduceOperatorBase reducer = (ReduceOperatorBase) sink.getInput(); - - // check types - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(initialData.getType()); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(initialData.getType()); - - // parallelism was not configured on the operator - assertThat(reducer.getParallelism()) - .isIn(parallelism, ExecutionConfig.PARALLELISM_DEFAULT); - - // check keys - assertThat(reducer.getKeyColumns(0)).containsExactly(2); - - assertThat(reducer.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateGroupedReduceWithkeyExtractor() { - try { - final int parallelism = 8; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> initialData = getSourceDataSet(env); - - initialData - .groupBy( - new KeySelector, StringValue>() { - public StringValue getKey( - Tuple3 value) { - return value.f1; - } - }) - .reduce( - new RichReduceFunction>() { - public Tuple3 reduce( - Tuple3 value1, - Tuple3 value2) { - return value1; - } - }) - .setParallelism(4) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - - MapOperatorBase keyProjector = (MapOperatorBase) sink.getInput(); - PlanUnwrappingReduceOperator reducer = - (PlanUnwrappingReduceOperator) keyProjector.getInput(); - MapOperatorBase keyExtractor = (MapOperatorBase) reducer.getInput(); - - // check the parallelisms - assertThat(keyExtractor.getParallelism()).isOne(); - assertThat(reducer.getParallelism()).isEqualTo(4); - assertThat(keyProjector.getParallelism()).isEqualTo(4); - - // check types - TypeInformation keyValueInfo = - new TupleTypeInfo>>( - new ValueTypeInfo(StringValue.class), - initialData.getType()); - - assertThat(keyExtractor.getOperatorInfo().getInputType()) - .isEqualTo(initialData.getType()); - assertThat(keyExtractor.getOperatorInfo().getOutputType()).isEqualTo(keyValueInfo); - - assertThat(reducer.getOperatorInfo().getInputType()).isEqualTo(keyValueInfo); - assertThat(reducer.getOperatorInfo().getOutputType()).isEqualTo(keyValueInfo); - - assertThat(keyProjector.getOperatorInfo().getInputType()).isEqualTo(keyValueInfo); - assertThat(keyProjector.getOperatorInfo().getOutputType()) - .isEqualTo(initialData.getType()); - - // check keys - assertThat(keyExtractor.getUserCodeWrapper().getUserCodeClass()) - .isEqualTo(KeyExtractingMapper.class); - - assertThat(keyExtractor.getInput()).isInstanceOf(GenericDataSourceBase.class); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @SuppressWarnings("unchecked") - private static DataSet> getSourceDataSet( - ExecutionEnvironment env) { - return env.fromElements( - new Tuple3<>(3.141592, new StringValue("foobar"), new LongValue(77))) - .setParallelism(1); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/UnionTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/UnionTranslationTest.java deleted file mode 100644 index 4db76ba86afbe..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/UnionTranslationTest.java +++ /dev/null @@ -1,168 +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.api.java.operators.translation; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.Union; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -/** Tests for translation of union operation. */ -@SuppressWarnings("serial") -class UnionTranslationTest { - - @Test - void translateUnion2Group() { - try { - final int parallelism = 4; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> dataset1 = getSourceDataSet(env, 3); - - DataSet> dataset2 = getSourceDataSet(env, 2); - - dataset1.union(dataset2) - .groupBy( - (KeySelector, String>) - value -> "") - .reduceGroup( - (GroupReduceFunction, String>) - (values, out) -> {}) - .returns(String.class) - .output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - // The plan should look like the following one. - // - // DataSet1(3) - MapOperator(3)-+ - // |- Union(-1) - SingleInputOperator - Sink - // DataSet2(2) - MapOperator(2)-+ - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - Union unionOperator = (Union) ((SingleInputOperator) sink.getInput()).getInput(); - - // The key mappers should be added to both of the two input streams for union. - assertThat(unionOperator.getFirstInput()).isInstanceOf(MapOperatorBase.class); - assertThat(unionOperator.getSecondInput()).isInstanceOf(MapOperatorBase.class); - - // The parallelisms of the key mappers should be equal to those of their inputs. - assertThat(unionOperator.getFirstInput().getParallelism()).isEqualTo(3); - assertThat(unionOperator.getSecondInput().getParallelism()).isEqualTo(2); - - // The union should always have the default parallelism. - assertThat(unionOperator.getParallelism()) - .isEqualTo(ExecutionConfig.PARALLELISM_DEFAULT); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @Test - void translateUnion3SortedGroup() { - try { - final int parallelism = 4; - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(parallelism); - - DataSet> dataset1 = getSourceDataSet(env, 2); - - DataSet> dataset2 = getSourceDataSet(env, 3); - - DataSet> dataset3 = getSourceDataSet(env, -1); - - dataset1.union(dataset2) - .union(dataset3) - .groupBy( - (KeySelector, String>) - value -> "") - .sortGroup( - (KeySelector, String>) - value -> "", - Order.ASCENDING) - .reduceGroup( - (GroupReduceFunction, String>) - (values, out) -> {}) - .returns(String.class) - .output(new DiscardingOutputFormat<>()); - - Plan p = env.createProgramPlan(); - - // The plan should look like the following one. - // - // DataSet1(2) - MapOperator(2)-+ - // |- Union(-1) -+ - // DataSet2(3) - MapOperator(3)-+ |- Union(-1) - SingleInputOperator - Sink - // | - // DataSet3(-1) - MapOperator(-1)-+ - - GenericDataSinkBase sink = p.getDataSinks().iterator().next(); - Union secondUnionOperator = (Union) ((SingleInputOperator) sink.getInput()).getInput(); - - // The first input of the second union should be the first union. - Union firstUnionOperator = (Union) secondUnionOperator.getFirstInput(); - - // The key mapper should be added to the second input stream of the second union. - assertThat(secondUnionOperator.getSecondInput()).isInstanceOf(MapOperatorBase.class); - - // The key mappers should be added to both of the two input streams for the first union. - assertThat(firstUnionOperator.getFirstInput()).isInstanceOf(MapOperatorBase.class); - assertThat(firstUnionOperator.getSecondInput()).isInstanceOf(MapOperatorBase.class); - - // The parallelisms of the key mappers should be equal to those of their inputs. - assertThat(firstUnionOperator.getFirstInput().getParallelism()).isEqualTo(2); - assertThat(firstUnionOperator.getSecondInput().getParallelism()).isEqualTo(3); - assertThat(secondUnionOperator.getSecondInput().getParallelism()).isEqualTo(-1); - - // The union should always have the default parallelism. - assertThat(secondUnionOperator.getParallelism()) - .isEqualTo(ExecutionConfig.PARALLELISM_DEFAULT); - assertThat(firstUnionOperator.getParallelism()) - .isEqualTo(ExecutionConfig.PARALLELISM_DEFAULT); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test caused an error: " + e.getMessage()); - } - } - - @SuppressWarnings("unchecked") - private static DataSet> getSourceDataSet( - ExecutionEnvironment env, int parallelism) { - return env.fromElements(new Tuple3<>(0.0, new StringValue(""), new LongValue(1L))) - .setParallelism(parallelism); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java deleted file mode 100644 index f1bae7b6a440c..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java +++ /dev/null @@ -1,504 +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.api.java.sampling; - -import org.apache.flink.testutils.junit.RetryOnFailure; -import org.apache.flink.testutils.junit.extensions.retry.RetryExtension; -import org.apache.flink.util.Preconditions; - -import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Set; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** - * This test suite try to verify whether all the random samplers work as we expected, which mainly - * focus on: - * - *
    - *
  • Does sampled result fit into input parameters? we check parameters like sample fraction, - * sample size, w/o replacement, and so on. - *
  • Does sampled result randomly selected? we verify this by measure how much does it - * distributed on source data. Run Kolmogorov-Smirnov (KS) test between the random samplers - * and default reference samplers which is distributed well-proportioned on source data. If - * random sampler select elements randomly from source, it would distributed well-proportioned - * on source data as well. The KS test will fail to strongly reject the null hypothesis that - * the distributions of sampling gaps are the same. - *
- * - * @see Kolmogorov Smirnov - * test - */ -@ExtendWith(RetryExtension.class) -class RandomSamplerTest { - - private static final int SOURCE_SIZE = 10000; - - private static final int DEFAULT_PARTITION_NUMBER = 10; - - private static final KolmogorovSmirnovTest ksTest = new KolmogorovSmirnovTest(); - - private static final List source = new ArrayList<>(SOURCE_SIZE); - - @SuppressWarnings({"unchecked", "rawtypes"}) - private final List[] sourcePartitions = new List[DEFAULT_PARTITION_NUMBER]; - - @BeforeAll - static void init() { - // initiate source data set. - for (int i = 0; i < SOURCE_SIZE; i++) { - source.add((double) i); - } - } - - private void initSourcePartition() { - for (int i = 0; i < DEFAULT_PARTITION_NUMBER; i++) { - sourcePartitions[i] = - new ArrayList<>( - (int) Math.ceil((double) SOURCE_SIZE / DEFAULT_PARTITION_NUMBER)); - } - for (int i = 0; i < SOURCE_SIZE; i++) { - int index = i % DEFAULT_PARTITION_NUMBER; - sourcePartitions[index].add((double) i); - } - } - - @Test - void testBernoulliSamplerWithUnexpectedFraction1() { - assertThatThrownBy(() -> verifySamplerFraction(-1, false)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testBernoulliSamplerWithUnexpectedFraction2() { - assertThatThrownBy(() -> verifySamplerFraction(2, false)) - .isInstanceOf(IllegalArgumentException.class); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testBernoulliSamplerFraction() { - verifySamplerFraction(0.01, false); - verifySamplerFraction(0.05, false); - verifySamplerFraction(0.1, false); - verifySamplerFraction(0.3, false); - verifySamplerFraction(0.5, false); - verifySamplerFraction(0.854, false); - verifySamplerFraction(0.99, false); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testBernoulliSamplerDuplicateElements() { - verifyRandomSamplerDuplicateElements(new BernoulliSampler<>(0.01)); - verifyRandomSamplerDuplicateElements(new BernoulliSampler<>(0.1)); - verifyRandomSamplerDuplicateElements(new BernoulliSampler<>(0.5)); - } - - @Test - void testPoissonSamplerWithUnexpectedFraction1() { - assertThatThrownBy(() -> verifySamplerFraction(-1, true)) - .isInstanceOf(IllegalArgumentException.class); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testPoissonSamplerFraction() { - verifySamplerFraction(0.01, true); - verifySamplerFraction(0.05, true); - verifySamplerFraction(0.1, true); - verifySamplerFraction(0.5, true); - verifySamplerFraction(0.854, true); - verifySamplerFraction(0.99, true); - verifySamplerFraction(1.5, true); - } - - @Test - void testReservoirSamplerUnexpectedSize1() { - assertThatThrownBy(() -> verifySamplerFixedSampleSize(-1, true)) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - void testReservoirSamplerUnexpectedSize2() { - assertThatThrownBy(() -> verifySamplerFixedSampleSize(-1, false)) - .isInstanceOf(IllegalArgumentException.class); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testBernoulliSamplerDistribution() { - verifyBernoulliSampler(0.01d); - verifyBernoulliSampler(0.05d); - verifyBernoulliSampler(0.1d); - verifyBernoulliSampler(0.5d); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testPoissonSamplerDistribution() { - verifyPoissonSampler(0.01d); - verifyPoissonSampler(0.05d); - verifyPoissonSampler(0.1d); - verifyPoissonSampler(0.5d); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerSampledSize() { - verifySamplerFixedSampleSize(1, true); - verifySamplerFixedSampleSize(10, true); - verifySamplerFixedSampleSize(100, true); - verifySamplerFixedSampleSize(1234, true); - verifySamplerFixedSampleSize(9999, true); - verifySamplerFixedSampleSize(20000, true); - - verifySamplerFixedSampleSize(1, false); - verifySamplerFixedSampleSize(10, false); - verifySamplerFixedSampleSize(100, false); - verifySamplerFixedSampleSize(1234, false); - verifySamplerFixedSampleSize(9999, false); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerSampledSize2() { - RandomSampler sampler = new ReservoirSamplerWithoutReplacement<>(20000); - Iterator sampled = sampler.sample(source.iterator()); - assertThat(getSize(sampled)) - .as( - "ReservoirSamplerWithoutReplacement sampled output size should not beyond the source size.") - .isEqualTo(SOURCE_SIZE); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerDuplicateElements() { - verifyRandomSamplerDuplicateElements(new ReservoirSamplerWithoutReplacement<>(100)); - verifyRandomSamplerDuplicateElements(new ReservoirSamplerWithoutReplacement<>(1000)); - verifyRandomSamplerDuplicateElements(new ReservoirSamplerWithoutReplacement<>(5000)); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerWithoutReplacement() { - verifyReservoirSamplerWithoutReplacement(100, false); - verifyReservoirSamplerWithoutReplacement(500, false); - verifyReservoirSamplerWithoutReplacement(1000, false); - verifyReservoirSamplerWithoutReplacement(5000, false); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerWithReplacement() { - verifyReservoirSamplerWithReplacement(100, false); - verifyReservoirSamplerWithReplacement(500, false); - verifyReservoirSamplerWithReplacement(1000, false); - verifyReservoirSamplerWithReplacement(5000, false); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerWithMultiSourcePartitions1() { - initSourcePartition(); - - verifyReservoirSamplerWithoutReplacement(100, true); - verifyReservoirSamplerWithoutReplacement(500, true); - verifyReservoirSamplerWithoutReplacement(1000, true); - verifyReservoirSamplerWithoutReplacement(5000, true); - } - - @TestTemplate - @RetryOnFailure(times = 3) - void testReservoirSamplerWithMultiSourcePartitions2() { - initSourcePartition(); - - verifyReservoirSamplerWithReplacement(100, true); - verifyReservoirSamplerWithReplacement(500, true); - verifyReservoirSamplerWithReplacement(1000, true); - verifyReservoirSamplerWithReplacement(5000, true); - } - - /* - * Sample with fixed size, verify whether the sampled result size equals to input size. - */ - private void verifySamplerFixedSampleSize(int numSample, boolean withReplacement) { - RandomSampler sampler; - if (withReplacement) { - sampler = new ReservoirSamplerWithReplacement<>(numSample); - } else { - sampler = new ReservoirSamplerWithoutReplacement<>(numSample); - } - Iterator sampled = sampler.sample(source.iterator()); - assertThat(getSize(sampled)).isEqualTo(numSample); - } - - /* - * Sample with fraction, and verify whether the sampled result close to input fraction. - */ - private void verifySamplerFraction(double fraction, boolean withReplacement) { - RandomSampler sampler; - if (withReplacement) { - sampler = new PoissonSampler<>(fraction); - } else { - sampler = new BernoulliSampler<>(fraction); - } - - // take 20 times sample, and take the average result size for next step comparison. - int totalSampledSize = 0; - double sampleCount = 20; - for (int i = 0; i < sampleCount; i++) { - totalSampledSize += getSize(sampler.sample(source.iterator())); - } - double resultFraction = totalSampledSize / ((double) SOURCE_SIZE * sampleCount); - assertThat(Math.abs((resultFraction - fraction) / fraction)) - .as( - String.format( - "expected fraction: %f, result fraction: %f", - fraction, resultFraction)) - .isLessThan(0.2); - } - - /* - * Test sampler without replacement, and verify that there should not exist any duplicate element in sampled result. - */ - private void verifyRandomSamplerDuplicateElements(final RandomSampler sampler) { - Iterator values = sampler.sample(source.iterator()); - Set set = new HashSet<>(); - while (values.hasNext()) { - double next = values.next(); - assertThat(set.add(next)) - .as("Sampler returned duplicate element (" + next + "). Set=" + set) - .isTrue(); - } - } - - private int getSize(Iterator iterator) { - int size = 0; - while (iterator.hasNext()) { - iterator.next(); - size++; - } - return size; - } - - private void verifyBernoulliSampler(double fraction) { - BernoulliSampler sampler = new BernoulliSampler<>(fraction); - verifyRandomSamplerWithFraction(fraction, sampler, true); - verifyRandomSamplerWithFraction(fraction, sampler, false); - } - - private void verifyPoissonSampler(double fraction) { - PoissonSampler sampler = new PoissonSampler<>(fraction); - verifyRandomSamplerWithFraction(fraction, sampler, true); - verifyRandomSamplerWithFraction(fraction, sampler, false); - } - - private void verifyReservoirSamplerWithReplacement( - int numSamplers, boolean sampleOnPartitions) { - ReservoirSamplerWithReplacement sampler = - new ReservoirSamplerWithReplacement<>(numSamplers); - verifyRandomSamplerWithSampleSize(numSamplers, sampler, true, sampleOnPartitions); - verifyRandomSamplerWithSampleSize(numSamplers, sampler, false, sampleOnPartitions); - } - - private void verifyReservoirSamplerWithoutReplacement( - int numSamplers, boolean sampleOnPartitions) { - ReservoirSamplerWithoutReplacement sampler = - new ReservoirSamplerWithoutReplacement<>(numSamplers); - verifyRandomSamplerWithSampleSize(numSamplers, sampler, true, sampleOnPartitions); - verifyRandomSamplerWithSampleSize(numSamplers, sampler, false, sampleOnPartitions); - } - - /* - * Verify whether random sampler sample with fraction from source data randomly. There are two default sample, one is - * sampled from source data with certain interval, the other is sampled only from the first half region of source data, - * If random sampler select elements randomly from source, it would distributed well-proportioned on source data as well, - * so the K-S Test result would accept the first one, while reject the second one. - */ - private void verifyRandomSamplerWithFraction( - double fraction, RandomSampler sampler, boolean withDefaultSampler) { - double[] baseSample; - if (withDefaultSampler) { - baseSample = getDefaultSampler(fraction); - } else { - baseSample = getWrongSampler(fraction); - } - - verifyKSTest(sampler, baseSample, withDefaultSampler); - } - - /* - * Verify whether random sampler sample with fixed size from source data randomly. There are two default sample, one is - * sampled from source data with certain interval, the other is sampled only from the first half region of source data, - * If random sampler select elements randomly from source, it would distributed well-proportioned on source data as well, - * so the K-S Test result would accept the first one, while reject the second one. - */ - private void verifyRandomSamplerWithSampleSize( - int sampleSize, - RandomSampler sampler, - boolean withDefaultSampler, - boolean sampleWithPartitions) { - double[] baseSample; - if (withDefaultSampler) { - baseSample = getDefaultSampler(sampleSize); - } else { - baseSample = getWrongSampler(sampleSize); - } - - verifyKSTest(sampler, baseSample, withDefaultSampler, sampleWithPartitions); - } - - private void verifyKSTest( - RandomSampler sampler, double[] defaultSampler, boolean expectSuccess) { - verifyKSTest(sampler, defaultSampler, expectSuccess, false); - } - - private void verifyKSTest( - RandomSampler sampler, - double[] defaultSampler, - boolean expectSuccess, - boolean sampleOnPartitions) { - double[] sampled = getSampledOutput(sampler, sampleOnPartitions); - double pValue = ksTest.kolmogorovSmirnovStatistic(sampled, defaultSampler); - double dValue = getDValue(sampled.length, defaultSampler.length); - if (expectSuccess) { - assertThat(pValue) - .as( - String.format( - "KS test result with p value(%f), d value(%f)", pValue, dValue)) - .isLessThanOrEqualTo(dValue); - } else { - assertThat(pValue) - .as( - String.format( - "KS test result with p value(%f), d value(%f)", pValue, dValue)) - .isGreaterThan(dValue); - } - } - - private double[] getSampledOutput(RandomSampler sampler, boolean sampleOnPartitions) { - Iterator sampled; - if (sampleOnPartitions) { - DistributedRandomSampler reservoirRandomSampler = - (DistributedRandomSampler) sampler; - List> intermediateResult = new LinkedList<>(); - for (int i = 0; i < DEFAULT_PARTITION_NUMBER; i++) { - Iterator> partialIntermediateResult = - reservoirRandomSampler.sampleInPartition(sourcePartitions[i].iterator()); - while (partialIntermediateResult.hasNext()) { - intermediateResult.add(partialIntermediateResult.next()); - } - } - sampled = reservoirRandomSampler.sampleInCoordinator(intermediateResult.iterator()); - } else { - sampled = sampler.sample(source.iterator()); - } - List list = new ArrayList<>(); - while (sampled.hasNext()) { - list.add(sampled.next()); - } - return transferFromListToArrayWithOrder(list); - } - - /* - * Some sample result may not order by the input sequence, we should make it in order to do K-S test. - */ - private double[] transferFromListToArrayWithOrder(List list) { - Collections.sort(list); - double[] result = new double[list.size()]; - for (int i = 0; i < list.size(); i++) { - result[i] = list.get(i); - } - return result; - } - - private double[] getDefaultSampler(double fraction) { - Preconditions.checkArgument(fraction > 0, "Sample fraction should be positive."); - int size = (int) (SOURCE_SIZE * fraction); - double step = 1 / fraction; - double[] defaultSampler = new double[size]; - for (int i = 0; i < size; i++) { - defaultSampler[i] = Math.round(step * i); - } - - return defaultSampler; - } - - private double[] getDefaultSampler(int fixSize) { - Preconditions.checkArgument(fixSize > 0, "Sample fraction should be positive."); - double step = SOURCE_SIZE / (double) fixSize; - double[] defaultSampler = new double[fixSize]; - for (int i = 0; i < fixSize; i++) { - defaultSampler[i] = Math.round(step * i); - } - - return defaultSampler; - } - - /* - * Build a failed sample distribution which only contains elements in the first half of source data. - */ - private double[] getWrongSampler(double fraction) { - Preconditions.checkArgument(fraction > 0, "Sample size should be positive."); - int size = (int) (SOURCE_SIZE * fraction); - int halfSourceSize = SOURCE_SIZE / 2; - double[] wrongSampler = new double[size]; - for (int i = 0; i < size; i++) { - wrongSampler[i] = (double) i % halfSourceSize; - } - - return wrongSampler; - } - - /* - * Build a failed sample distribution which only contains elements in the first half of source data. - */ - private double[] getWrongSampler(int fixSize) { - Preconditions.checkArgument(fixSize > 0, "Sample size be positive."); - int halfSourceSize = SOURCE_SIZE / 2; - double[] wrongSampler = new double[fixSize]; - for (int i = 0; i < fixSize; i++) { - wrongSampler[i] = (double) i % halfSourceSize; - } - - return wrongSampler; - } - - /* - * Calculate the D value of K-S test for p-value 0.001, m and n are the sample size - */ - private double getDValue(int m, int n) { - Preconditions.checkArgument(m > 0, "input sample size should be positive."); - Preconditions.checkArgument(n > 0, "input sample size should be positive."); - return 1.95 * Math.sqrt(((double) m + (double) n) / ((double) m * (double) n)); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java deleted file mode 100644 index 12da3c272d8eb..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java +++ /dev/null @@ -1,112 +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.api.java.summarize.aggregation; - -import java.lang.reflect.ParameterizedType; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -/** - * This harness uses multiple aggregators and variously aggregates and combines against a list of - * values while calling a compareResults() method. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a bunch - * of different ways but can help uncover various kinds of bugs that can show up in aggregators. - * - * @param the type to aggregate - * @param the type of the results of the aggregation - * @param the aggregator to use - */ -public abstract class AggregateCombineHarness> { - - /** - * Compare results from different runs of aggregate/combine to make sure they are the same. - * - *

Subclasses should cause an Assertion failure or throw an Exception if the results aren't - * equal or at least close enough. - */ - protected abstract void compareResults(R result1, R result2); - - /** Variously aggregate and combine against a list of values, comparing results each time. */ - @SafeVarargs - public final R summarize(T... values) { - if (values.length == 0) { - // when there is nothing to aggregate just combine two empty aggregators and get the - // result. - A agg1 = initAggregator(); - agg1.combine(initAggregator()); - return agg1.result(); - } else { - R previousResult = null; - R result = null; - - // Shuffling the values might cause test instability but only in the - // case that there are underlying bugs that need to be fixed - List list = Arrays.asList(values); - Collections.shuffle(list); - - for (int i = 0; i < values.length; i++) { - - // Two aggregators are used so that combine() can be tested also. - // It shouldn't matter which aggregator is used because they are combined at the end - // so - // we're looping through all points of the data and making sure it doesn't make a - // difference. - - A aggregator1 = initAggregator(); - A aggregator2 = initAggregator(); - - for (int j = 0; j < i; j++) { - aggregator1.aggregate(list.get(j)); - } - for (int j = i; j < values.length; j++) { - aggregator2.aggregate(list.get(j)); - } - - aggregator1.combine(aggregator2); - - previousResult = result; - result = aggregator1.result(); - - if (previousResult != null) { - // validate that variously aggregating then combining doesn't give different - // results - compareResults(result, previousResult); - } - } - return result; - } - } - - @SuppressWarnings("unchecked") - private A initAggregator() { - try { - // Instantiate a generic type - // http://stackoverflow.com/questions/75175/create-instance-of-generic-type-in-java - return (A) - ((Class) - ((ParameterizedType) this.getClass().getGenericSuperclass()) - .getActualTypeArguments()[2]) - .newInstance(); - } catch (Exception e) { - throw new RuntimeException("Could not initialize aggregator", e); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java deleted file mode 100644 index 736c6b823f93d..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java +++ /dev/null @@ -1,90 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.BooleanColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link BooleanSummaryAggregator}. */ -class BooleanSummaryAggregatorTest { - - @Test - void testMixedGroup() { - BooleanColumnSummary summary = - summarize(true, false, null, true, true, true, false, null, true, false, true); - assertThat(summary.getTotalCount()).isEqualTo(11); - assertThat(summary.getNullCount()).isEqualTo(2); - assertThat(summary.getNonNullCount()).isEqualTo(9); - assertThat(summary.getTrueCount()).isEqualTo(6); - assertThat(summary.getFalseCount()).isEqualTo(3); - } - - @Test - void testAllNullBooleans() { - BooleanColumnSummary summary = summarize(null, null, null, null); - assertThat(summary.getTotalCount()).isEqualTo(4); - assertThat(summary.getNullCount()).isEqualTo(4); - assertThat(summary.getNonNullCount()).isZero(); - assertThat(summary.getTrueCount()).isZero(); - assertThat(summary.getFalseCount()).isZero(); - } - - @Test - void testAllTrue() { - BooleanColumnSummary summary = summarize(true, true, true, true, true, true); - assertThat(summary.getTotalCount()).isEqualTo(6); - assertThat(summary.getNullCount()).isZero(); - assertThat(summary.getNonNullCount()).isEqualTo(6); - assertThat(summary.getTrueCount()).isEqualTo(6); - assertThat(summary.getFalseCount()).isZero(); - } - - @Test - void testAllFalse() { - BooleanColumnSummary summary = summarize(false, false, false); - assertThat(summary.getTotalCount()).isEqualTo(3); - assertThat(summary.getNullCount()).isZero(); - assertThat(summary.getNonNullCount()).isEqualTo(3); - assertThat(summary.getTrueCount()).isZero(); - assertThat(summary.getFalseCount()).isEqualTo(3); - } - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - protected BooleanColumnSummary summarize(Boolean... values) { - return new AggregateCombineHarness< - Boolean, BooleanColumnSummary, BooleanSummaryAggregator>() { - @Override - protected void compareResults( - BooleanColumnSummary result1, BooleanColumnSummary result2) { - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getNonNullCount()).isEqualTo(result1.getNonNullCount()); - assertThat(result2.getTrueCount()).isEqualTo(result1.getTrueCount()); - assertThat(result2.getFalseCount()).isEqualTo(result1.getFalseCount()); - } - }.summarize(values); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java deleted file mode 100644 index cf333c647ab7e..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java +++ /dev/null @@ -1,58 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.BooleanColumnSummary; -import org.apache.flink.types.BooleanValue; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link ValueSummaryAggregator.BooleanValueSummaryAggregator}. */ -class BooleanValueSummaryAggregatorTest extends BooleanSummaryAggregatorTest { - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - protected BooleanColumnSummary summarize(Boolean... values) { - - BooleanValue[] booleanValues = new BooleanValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - booleanValues[i] = new BooleanValue(values[i]); - } - } - - return new AggregateCombineHarness< - BooleanValue, - BooleanColumnSummary, - ValueSummaryAggregator.BooleanValueSummaryAggregator>() { - @Override - protected void compareResults( - BooleanColumnSummary result1, BooleanColumnSummary result2) { - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getNonNullCount()).isEqualTo(result1.getNonNullCount()); - assertThat(result2.getTrueCount()).isEqualTo(result1.getTrueCount()); - assertThat(result2.getFalseCount()).isEqualTo(result1.getFalseCount()); - } - }.summarize(booleanValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java deleted file mode 100644 index 5832712587d2c..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java +++ /dev/null @@ -1,82 +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.api.java.summarize.aggregation; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link CompensatedSum}. */ -class CompensatedSumTest { - - /** - * When adding a series of numbers the order of the numbers should not impact the results. - * - *

This test shows that a naive summation comes up with a different result than Kahan - * Summation when you start with either a smaller or larger number in some cases and helps prove - * our Kahan Summation is working. - */ - @Test - void testAdd1() { - final CompensatedSum smallSum = new CompensatedSum(0.001, 0.0); - final CompensatedSum largeSum = new CompensatedSum(1000, 0.0); - - CompensatedSum compensatedResult1 = smallSum; - CompensatedSum compensatedResult2 = largeSum; - double naiveResult1 = smallSum.value(); - double naiveResult2 = largeSum.value(); - - for (int i = 0; i < 10; i++) { - compensatedResult1 = compensatedResult1.add(smallSum); - compensatedResult2 = compensatedResult2.add(smallSum); - naiveResult1 += smallSum.value(); - naiveResult2 += smallSum.value(); - } - - compensatedResult1 = compensatedResult1.add(largeSum); - compensatedResult2 = compensatedResult2.add(smallSum); - naiveResult1 += largeSum.value(); - naiveResult2 += smallSum.value(); - - // Kahan summation gave the same result no matter what order we added - assertThat(compensatedResult1.value()).isCloseTo(1000.011, offset(0.0)); - assertThat(compensatedResult2.value()).isCloseTo(1000.011, offset(0.0)); - - // naive addition gave a small floating point error - assertThat(naiveResult1).isCloseTo(1000.011, offset(0.0)); - assertThat(naiveResult2).isCloseTo(1000.0109999999997, offset(0.0)); - - assertThat(compensatedResult2.value()).isCloseTo(compensatedResult1.value(), offset(0.0)); - assertThat(naiveResult2) - .isCloseTo(naiveResult1, offset(0.0001)) - .isNotCloseTo(naiveResult1, offset(0.0)); - } - - @Test - void testDelta() { - CompensatedSum compensatedResult1 = new CompensatedSum(0.001, 0.0); - for (int i = 0; i < 10; i++) { - compensatedResult1 = compensatedResult1.add(0.001); - } - assertThat(compensatedResult1.value()).isCloseTo(0.011, offset(0.0)); - assertThat(compensatedResult1.delta()) - .isCloseTo(new Double("8.673617379884035E-19"), offset(0.0)); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java deleted file mode 100644 index d22ff4b5b714d..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java +++ /dev/null @@ -1,223 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link DoubleSummaryAggregator}. */ -class DoubleSummaryAggregatorTest { - - /** - * Use some values from Anscombe's Quartet for testing. - * - *

There was no particular reason to use these except they have known means and variance. - * - *

https://en.wikipedia.org/wiki/Anscombe%27s_quartet - */ - @Test - void testAnscomesQuartetXValues() { - - final Double[] q1x = {10.0, 8.0, 13.0, 9.0, 11.0, 14.0, 6.0, 4.0, 12.0, 7.0, 5.0}; - final Double[] q4x = {8.0, 8.0, 8.0, 8.0, 8.0, 8.0, 8.0, 19.0, 8.0, 8.0, 8.0}; - - NumericColumnSummary q1 = summarize(q1x); - NumericColumnSummary q4 = summarize(q4x); - - assertThat(q1.getMean().doubleValue()).isCloseTo(9.0, offset(0.0)); - assertThat(q4.getMean().doubleValue()).isCloseTo(9.0, offset(0.0)); - - assertThat(q1.getVariance().doubleValue()).isCloseTo(11.0, offset(1e-10d)); - assertThat(q4.getVariance().doubleValue()).isCloseTo(11.0, offset(1e-10d)); - - double stddev = Math.sqrt(11.0); - assertThat(q1.getStandardDeviation().doubleValue()).isCloseTo(stddev, offset(1e-10d)); - assertThat(q4.getStandardDeviation().doubleValue()).isCloseTo(stddev, offset(1e-10d)); - } - - /** - * Use some values from Anscombe's Quartet for testing. - * - *

There was no particular reason to use these except they have known means and variance. - * - *

https://en.wikipedia.org/wiki/Anscombe%27s_quartet - */ - @Test - void testAnscomesQuartetYValues() { - final Double[] q1y = {8.04, 6.95, 7.58, 8.81, 8.33, 9.96, 7.24, 4.26, 10.84, 4.82, 5.68}; - final Double[] q2y = {9.14, 8.14, 8.74, 8.77, 9.26, 8.1, 6.13, 3.1, 9.13, 7.26, 4.74}; - final Double[] q3y = {7.46, 6.77, 12.74, 7.11, 7.81, 8.84, 6.08, 5.39, 8.15, 6.42, 5.73}; - final Double[] q4y = {6.58, 5.76, 7.71, 8.84, 8.47, 7.04, 5.25, 12.5, 5.56, 7.91, 6.89}; - - NumericColumnSummary q1 = summarize(q1y); - NumericColumnSummary q2 = summarize(q2y); - NumericColumnSummary q3 = summarize(q3y); - NumericColumnSummary q4 = summarize(q4y); - - // the y values are have less precisely matching means and variances - - assertThat(q1.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - assertThat(q2.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - assertThat(q3.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - assertThat(q4.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - - assertThat(q1.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - assertThat(q2.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - assertThat(q3.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - assertThat(q4.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - } - - @Test - void testIsNan() { - DoubleSummaryAggregator ag = new DoubleSummaryAggregator(); - assertThat(ag.isNan(-1.0)).isFalse(); - assertThat(ag.isNan(0.0)).isFalse(); - assertThat(ag.isNan(23.0)).isFalse(); - assertThat(ag.isNan(Double.MAX_VALUE)).isFalse(); - assertThat(ag.isNan(Double.MIN_VALUE)).isFalse(); - assertThat(ag.isNan(Double.NaN)).isTrue(); - } - - @Test - void testIsInfinite() { - DoubleSummaryAggregator ag = new DoubleSummaryAggregator(); - assertThat(ag.isInfinite(-1.0)).isFalse(); - assertThat(ag.isInfinite(0.0)).isFalse(); - assertThat(ag.isInfinite(23.0)).isFalse(); - assertThat(ag.isInfinite(Double.MAX_VALUE)).isFalse(); - assertThat(ag.isInfinite(Double.MIN_VALUE)).isFalse(); - assertThat(ag.isInfinite(Double.POSITIVE_INFINITY)).isTrue(); - assertThat(ag.isInfinite(Double.NEGATIVE_INFINITY)).isTrue(); - } - - @Test - void testMean() { - assertThat(summarize(0.0, 100.0).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0.0, 0.0, 100.0).getMean()).isCloseTo(33.333333, offset(0.00001)); - assertThat(summarize(0.0, 0.0, 100.0, 100.0).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0.0, 100.0, null).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize().getMean()).isNull(); - } - - @Test - void testSum() throws Exception { - assertThat(summarize(0.0, 100.0).getSum().doubleValue()).isCloseTo(100.0, offset(0.0)); - assertThat(summarize(1.0, 2.0, 3.0, 4.0, 5.0).getSum().doubleValue()) - .isCloseTo(15, offset(0.0)); - assertThat(summarize(-100.0, 0.0, 100.0, null).getSum().doubleValue()) - .isCloseTo(0, offset(0.0)); - assertThat(summarize(-10.0, 100.0, null).getSum().doubleValue()).isCloseTo(90, offset(0.0)); - assertThat(summarize().getSum()).isNull(); - } - - @Test - void testMax() { - assertThat(summarize(-1000.0, 0.0, 1.0, 50.0, 999.0, 1001.0).getMax().doubleValue()) - .isCloseTo(1001.0, offset(0.0)); - assertThat( - summarize( - 1.0, 8.0, 7.0, 6.0, 9.0, 10.0, 2.0, 3.0, 5.0, 0.0, 11.0, - -2.0, 3.0) - .getMax() - .doubleValue()) - .isCloseTo(11.0, offset(0.0)); - assertThat( - summarize( - 1.0, 8.0, 7.0, 6.0, 9.0, null, 10.0, 2.0, 3.0, 5.0, null, - 0.0, 11.0, -2.0, 3.0) - .getMax() - .doubleValue()) - .isCloseTo(11.0, offset(0.0)); - assertThat(summarize(-1.0, -8.0, -7.0, null, -11.0).getMax().doubleValue()) - .isCloseTo(-1.0, offset(0.0)); - assertThat(summarize().getMax()).isNull(); - } - - @Test - void testMin() { - assertThat(summarize(-1000.0, 0.0, 1.0, 50.0, 999.0, 1001.0).getMin().doubleValue()) - .isCloseTo(-1000, offset(0.0)); - assertThat( - summarize( - 1.0, 8.0, 7.0, 6.0, 9.0, 10.0, 2.0, 3.0, 5.0, 0.0, 11.0, - -2.0, 3.0) - .getMin() - .doubleValue()) - .isCloseTo(-2.0, offset(0.0)); - assertThat( - summarize( - 1.0, 8.0, 7.0, 6.0, 9.0, null, 10.0, 2.0, 3.0, 5.0, null, - 0.0, 11.0, -2.0, 3.0) - .getMin() - .doubleValue()) - .isCloseTo(-2.0, offset(0.0)); - assertThat(summarize().getMin()).isNull(); - } - - @Test - void testCounts() { - NumericColumnSummary summary = - summarize( - Double.NaN, - 1.0, - null, - 123.0, - -44.00001, - Double.POSITIVE_INFINITY, - 55.0, - Double.NEGATIVE_INFINITY, - Double.NEGATIVE_INFINITY, - null, - Double.NaN); - assertThat(summary.getTotalCount()).isEqualTo(11); - assertThat(summary.getNullCount()).isEqualTo(2); - assertThat(summary.getNonNullCount()).isEqualTo(9); - assertThat(summary.getMissingCount()).isEqualTo(7); - assertThat(summary.getNonMissingCount()).isEqualTo(4); - assertThat(summary.getNanCount()).isEqualTo(2); - assertThat(summary.getInfinityCount()).isEqualTo(3); - } - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - protected NumericColumnSummary summarize(Double... values) { - return new AggregateCombineHarness< - Double, NumericColumnSummary, DoubleSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - assertThat(result2.getMin()).isCloseTo(result1.getMin(), offset(0.0)); - assertThat(result2.getMax()).isCloseTo(result1.getMax(), offset(0.0)); - assertThat(result2.getMean()).isCloseTo(result1.getMean(), offset(1e-12d)); - assertThat(result2.getVariance()).isCloseTo(result1.getVariance(), offset(1e-9d)); - assertThat(result2.getStandardDeviation()) - .isCloseTo(result1.getStandardDeviation(), offset(1e-12d)); - } - }.summarize(values); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java deleted file mode 100644 index f166c9c60677a..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; -import org.apache.flink.types.DoubleValue; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ValueSummaryAggregator.DoubleValueSummaryAggregator}. */ -class DoubleValueSummaryAggregatorTest extends DoubleSummaryAggregatorTest { - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - protected NumericColumnSummary summarize(Double... values) { - - DoubleValue[] doubleValues = new DoubleValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - doubleValues[i] = new DoubleValue(values[i]); - } - } - - return new AggregateCombineHarness< - DoubleValue, - NumericColumnSummary, - ValueSummaryAggregator.DoubleValueSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - assertThat(result2.getMin()).isCloseTo(result1.getMin(), offset(0.0)); - assertThat(result2.getMax()).isCloseTo(result1.getMax(), offset(0.0)); - assertThat(result2.getMean()).isCloseTo(result1.getMean(), offset(1e-12d)); - assertThat(result2.getVariance()).isCloseTo(result1.getVariance(), offset(1e-9d)); - assertThat(result2.getStandardDeviation()) - .isCloseTo(result1.getStandardDeviation(), offset(1e-12d)); - } - }.summarize(doubleValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java deleted file mode 100644 index c33e254b6662e..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.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.0f (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.0f - * - * 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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link FloatSummaryAggregator}. */ -class FloatSummaryAggregatorTest { - - /** - * Use some values from Anscombe's Quartet for testing. - * - *

There was no particular reason to use these except they have known means and variance. - * - *

https://en.wikipedia.org/wiki/Anscombe%27s_quartet - */ - @Test - void testAnscomesQuartetXValues() { - - final Float[] q1x = {10.0f, 8.0f, 13.0f, 9.0f, 11.0f, 14.0f, 6.0f, 4.0f, 12.0f, 7.0f, 5.0f}; - final Float[] q4x = {8.0f, 8.0f, 8.0f, 8.0f, 8.0f, 8.0f, 8.0f, 19.0f, 8.0f, 8.0f, 8.0f}; - - NumericColumnSummary q1 = summarize(q1x); - NumericColumnSummary q4 = summarize(q4x); - - assertThat(q1.getMean().doubleValue()).isCloseTo(9.0, offset(0.0)); - assertThat(q4.getMean().doubleValue()).isCloseTo(9.0, offset(0.0)); - - assertThat(q1.getVariance().doubleValue()).isCloseTo(11.0, offset(1e-10d)); - assertThat(q4.getVariance().doubleValue()).isCloseTo(11.0, offset(1e-10d)); - - double stddev = Math.sqrt(11.0f); - assertThat(q1.getStandardDeviation().doubleValue()).isCloseTo(stddev, offset(1e-10d)); - assertThat(q4.getStandardDeviation().doubleValue()).isCloseTo(stddev, offset(1e-10d)); - } - - /** - * Use some values from Anscombe's Quartet for testing. - * - *

There was no particular reason to use these except they have known means and variance. - * - *

https://en.wikipedia.org/wiki/Anscombe%27s_quartet - */ - @Test - void testAnscomesQuartetYValues() { - final Float[] q1y = { - 8.04f, 6.95f, 7.58f, 8.81f, 8.33f, 9.96f, 7.24f, 4.26f, 10.84f, 4.82f, 5.68f - }; - final Float[] q2y = { - 9.14f, 8.14f, 8.74f, 8.77f, 9.26f, 8.1f, 6.13f, 3.1f, 9.13f, 7.26f, 4.74f - }; - final Float[] q3y = { - 7.46f, 6.77f, 12.74f, 7.11f, 7.81f, 8.84f, 6.08f, 5.39f, 8.15f, 6.42f, 5.73f - }; - final Float[] q4y = { - 6.58f, 5.76f, 7.71f, 8.84f, 8.47f, 7.04f, 5.25f, 12.5f, 5.56f, 7.91f, 6.89f - }; - - NumericColumnSummary q1 = summarize(q1y); - NumericColumnSummary q2 = summarize(q2y); - NumericColumnSummary q3 = summarize(q3y); - NumericColumnSummary q4 = summarize(q4y); - - // the y values are have less precisely matching means and variances - - assertThat(q1.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - assertThat(q2.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - assertThat(q3.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - assertThat(q4.getMean().doubleValue()).isCloseTo(7.5, offset(0.001)); - - assertThat(q1.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - assertThat(q2.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - assertThat(q3.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - assertThat(q4.getVariance().doubleValue()).isCloseTo(4.12, offset(0.01)); - } - - @Test - void testIsNan() { - FloatSummaryAggregator ag = new FloatSummaryAggregator(); - assertThat(ag.isNan(-1.0f)).isFalse(); - assertThat(ag.isNan(0.0f)).isFalse(); - assertThat(ag.isNan(23.0f)).isFalse(); - assertThat(ag.isNan(Float.MAX_VALUE)).isFalse(); - assertThat(ag.isNan(Float.MIN_VALUE)).isFalse(); - assertThat(ag.isNan(Float.NaN)).isTrue(); - } - - @Test - void testIsInfinite() { - FloatSummaryAggregator ag = new FloatSummaryAggregator(); - assertThat(ag.isInfinite(-1.0f)).isFalse(); - assertThat(ag.isInfinite(0.0f)).isFalse(); - assertThat(ag.isInfinite(23.0f)).isFalse(); - assertThat(ag.isInfinite(Float.MAX_VALUE)).isFalse(); - assertThat(ag.isInfinite(Float.MIN_VALUE)).isFalse(); - assertThat(ag.isInfinite(Float.POSITIVE_INFINITY)).isTrue(); - assertThat(ag.isInfinite(Float.NEGATIVE_INFINITY)).isTrue(); - } - - @Test - void testMean() { - assertThat(summarize(0.0f, 100.0f).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0.0f, 0.0f, 100.0f).getMean()).isCloseTo(33.333333, offset(0.00001)); - assertThat(summarize(0.0f, 0.0f, 100.0f, 100.0f).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0.0f, 100.0f, null).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize().getMean()).isNull(); - } - - @Test - void testSum() throws Exception { - assertThat(summarize(0.0f, 100.0f).getSum().floatValue()).isCloseTo(100.0f, offset(0.0f)); - assertThat(summarize(1.0f, 2.0f, 3.0f, 4.0f, 5.0f).getSum().floatValue()) - .isCloseTo(15, offset(0.0f)); - assertThat(summarize(-100.0f, 0.0f, 100.0f, null).getSum().floatValue()) - .isCloseTo(0, offset(0.0f)); - assertThat(summarize(-10.0f, 100.0f, null).getSum().floatValue()) - .isCloseTo(90, offset(0.0f)); - assertThat(summarize().getSum()).isNull(); - } - - @Test - void testMax() { - assertThat(summarize(-1000.0f, 0.0f, 1.0f, 50.0f, 999.0f, 1001.0f).getMax().floatValue()) - .isCloseTo(1001.0f, offset(0.0f)); - assertThat( - summarize( - 1.0f, 8.0f, 7.0f, 6.0f, 9.0f, 10.0f, 2.0f, 3.0f, 5.0f, 0.0f, - 11.0f, -2.0f, 3.0f) - .getMax() - .floatValue()) - .isCloseTo(11.0f, offset(0.0f)); - assertThat( - summarize( - 1.0f, 8.0f, 7.0f, 6.0f, 9.0f, null, 10.0f, 2.0f, 3.0f, 5.0f, - null, 0.0f, 11.0f, -2.0f, 3.0f) - .getMax() - .floatValue()) - .isCloseTo(11.0f, offset(0.0f)); - assertThat( - summarize(-8.0f, -7.0f, -6.0f, -9.0f, null, -10.0f, null, -2.0f) - .getMax() - .floatValue()) - .isCloseTo(-2.0f, offset(0.0f)); - assertThat(summarize().getMax()).isNull(); - } - - @Test - void testMin() { - assertThat(summarize(-1000.0f, 0.0f, 1.0f, 50.0f, 999.0f, 1001.0f).getMin().floatValue()) - .isCloseTo(-1000, offset(0.0f)); - assertThat( - summarize( - 1.0f, 8.0f, 7.0f, 6.0f, 9.0f, 10.0f, 2.0f, 3.0f, 5.0f, 0.0f, - 11.0f, -2.0f, 3.0f) - .getMin() - .floatValue()) - .isCloseTo(-2.0f, offset(0.0f)); - assertThat( - summarize( - 1.0f, 8.0f, 7.0f, 6.0f, 9.0f, null, 10.0f, 2.0f, 3.0f, 5.0f, - null, 0.0f, 11.0f, -2.0f, 3.0f) - .getMin() - .floatValue()) - .isCloseTo(-2.0f, offset(0.0f)); - assertThat(summarize().getMin()).isNull(); - } - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - protected NumericColumnSummary summarize(Float... values) { - - return new AggregateCombineHarness< - Float, NumericColumnSummary, FloatSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - assertThat(result2.getMin()).isCloseTo(result1.getMin(), offset(0.0f)); - assertThat(result2.getMax()).isCloseTo(result1.getMax(), offset(0.0f)); - assertThat(result2.getMean()).isCloseTo(result1.getMean(), offset(1e-12d)); - assertThat(result2.getVariance()).isCloseTo(result1.getVariance(), offset(1e-9d)); - assertThat(result2.getStandardDeviation()) - .isCloseTo(result1.getStandardDeviation(), offset(1e-12d)); - } - }.summarize(values); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java deleted file mode 100644 index c9041f9e9c0b3..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java +++ /dev/null @@ -1,63 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; -import org.apache.flink.types.FloatValue; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ValueSummaryAggregator.FloatValueSummaryAggregator}. */ -class FloatValueSummaryAggregatorTest extends FloatSummaryAggregatorTest { - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - @Override - protected NumericColumnSummary summarize(Float... values) { - - FloatValue[] floatValues = new FloatValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - floatValues[i] = new FloatValue(values[i]); - } - } - - return new AggregateCombineHarness< - FloatValue, - NumericColumnSummary, - ValueSummaryAggregator.FloatValueSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - assertThat(result2.getMin()).isCloseTo(result1.getMin(), offset(0.0f)); - assertThat(result2.getMax()).isCloseTo(result1.getMax(), offset(0.0f)); - assertThat(result2.getMean()).isCloseTo(result1.getMean(), offset(1e-10d)); - assertThat(result2.getVariance()).isCloseTo(result1.getVariance(), offset(1e-9d)); - assertThat(result2.getStandardDeviation()) - .isCloseTo(result1.getStandardDeviation(), offset(1e-10d)); - } - }.summarize(floatValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java deleted file mode 100644 index 8d6a548866982..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java +++ /dev/null @@ -1,134 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link IntegerSummaryAggregator}. */ -class IntegerSummaryAggregatorTest { - - @Test - void testIsNan() { - IntegerSummaryAggregator ag = new IntegerSummaryAggregator(); - // always false for Integer - assertThat(ag.isNan(-1)).isFalse(); - assertThat(ag.isNan(0)).isFalse(); - assertThat(ag.isNan(23)).isFalse(); - assertThat(ag.isNan(Integer.MAX_VALUE)).isFalse(); - assertThat(ag.isNan(Integer.MIN_VALUE)).isFalse(); - assertThat(ag.isNan(null)).isFalse(); - } - - @Test - void testIsInfinite() { - IntegerSummaryAggregator ag = new IntegerSummaryAggregator(); - // always false for Integer - assertThat(ag.isInfinite(-1)).isFalse(); - assertThat(ag.isInfinite(0)).isFalse(); - assertThat(ag.isInfinite(23)).isFalse(); - assertThat(ag.isInfinite(Integer.MAX_VALUE)).isFalse(); - assertThat(ag.isInfinite(Integer.MIN_VALUE)).isFalse(); - assertThat(ag.isInfinite(null)).isFalse(); - } - - @Test - void testMean() { - assertThat(summarize(0, 100).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0, 0, 100).getMean()).isCloseTo(33.333333, offset(0.00001)); - assertThat(summarize(0, 0, 100, 100).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0, 100, null).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize().getMean()).isNull(); - } - - @Test - void testSum() throws Exception { - assertThat(summarize(0, 100).getSum().intValue()).isEqualTo(100); - assertThat(summarize(1, 2, 3, 4, 5).getSum().intValue()).isEqualTo(15); - assertThat(summarize(-100, 0, 100, null).getSum().intValue()).isZero(); - assertThat(summarize(-10, 100, null).getSum().intValue()).isEqualTo(90); - assertThat(summarize().getSum()).isNull(); - } - - @Test - void testMax() { - assertThat(summarize(-1000, 0, 1, 50, 999, 1001).getMax().intValue()).isEqualTo(1001); - assertThat(summarize(Integer.MIN_VALUE, -1000, 0).getMax().intValue()).isZero(); - assertThat(summarize(1, 8, 7, 6, 9, 10, 2, 3, 5, 0, 11, -2, 3).getMax().intValue()) - .isEqualTo(11); - assertThat( - summarize(1, 8, 7, 6, 9, null, 10, 2, 3, 5, null, 0, 11, -2, 3) - .getMax() - .intValue()) - .isEqualTo(11); - assertThat(summarize().getMax()).isNull(); - } - - @Test - void testMin() { - assertThat(summarize(-1000, 0, 1, 50, 999, 1001).getMin().intValue()).isEqualTo(-1000); - assertThat(summarize(Integer.MIN_VALUE, -1000, 0).getMin().intValue()) - .isEqualTo(Integer.MIN_VALUE); - assertThat(summarize(1, 8, 7, 6, 9, 10, 2, 3, 5, 0, 11, -2, 3).getMin().intValue()) - .isEqualTo(-2); - assertThat( - summarize(1, 8, 7, 6, 9, null, 10, 2, 3, 5, null, 0, 11, -2, 3) - .getMin() - .intValue()) - .isEqualTo(-2); - assertThat(summarize().getMin()).isNull(); - } - - /** Helper method for summarizing a list of values. */ - protected NumericColumnSummary summarize(Integer... values) { - - return new AggregateCombineHarness< - Integer, NumericColumnSummary, IntegerSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - - assertThat(result2.getTotalCount()).isEqualTo(result1.getTotalCount()); - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getMissingCount()).isEqualTo(result1.getMissingCount()); - assertThat(result2.getNonMissingCount()).isEqualTo(result1.getNonMissingCount()); - assertThat(result2.getInfinityCount()).isEqualTo(result1.getInfinityCount()); - assertThat(result2.getNanCount()).isEqualTo(result1.getNanCount()); - - assertThat(result2.containsNull()).isEqualTo(result1.containsNull()); - assertThat(result2.containsNonNull()).isEqualTo(result1.containsNonNull()); - - assertThat(result2.getMin().intValue()).isEqualTo(result1.getMin().intValue()); - assertThat(result2.getMax().intValue()).isEqualTo(result1.getMax().intValue()); - assertThat(result2.getSum().intValue()).isEqualTo(result1.getSum().intValue()); - assertThat(result2.getMean().doubleValue()) - .isCloseTo(result1.getMean().doubleValue(), offset(1e-12d)); - assertThat(result2.getVariance().doubleValue()) - .isCloseTo(result1.getVariance().doubleValue(), offset(1e-9d)); - assertThat(result2.getStandardDeviation().doubleValue()) - .isCloseTo(result1.getStandardDeviation().doubleValue(), offset(1e-12d)); - } - }.summarize(values); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java deleted file mode 100644 index eace60a4c84ca..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java +++ /dev/null @@ -1,71 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; -import org.apache.flink.types.IntValue; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ValueSummaryAggregator.IntegerValueSummaryAggregator}. */ -class IntegerValueSummaryAggregatorTest extends IntegerSummaryAggregatorTest { - - @Override - protected NumericColumnSummary summarize(Integer... values) { - - IntValue[] intValues = new IntValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - intValues[i] = new IntValue(values[i]); - } - } - - return new AggregateCombineHarness< - IntValue, - NumericColumnSummary, - ValueSummaryAggregator.IntegerValueSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - - assertThat(result2.getTotalCount()).isEqualTo(result1.getTotalCount()); - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getMissingCount()).isEqualTo(result1.getMissingCount()); - assertThat(result2.getNonMissingCount()).isEqualTo(result1.getNonMissingCount()); - assertThat(result2.getInfinityCount()).isEqualTo(result1.getInfinityCount()); - assertThat(result2.getNanCount()).isEqualTo(result1.getNanCount()); - - assertThat(result2.containsNull()).isEqualTo(result1.containsNull()); - assertThat(result2.containsNonNull()).isEqualTo(result1.containsNonNull()); - - assertThat(result2.getMin().intValue()).isEqualTo(result1.getMin().intValue()); - assertThat(result2.getMax().intValue()).isEqualTo(result1.getMax().intValue()); - assertThat(result2.getSum().intValue()).isEqualTo(result1.getSum().intValue()); - assertThat(result2.getMean().doubleValue()) - .isCloseTo(result1.getMean().doubleValue(), offset(1e-12d)); - assertThat(result2.getVariance().doubleValue()) - .isCloseTo(result1.getVariance().doubleValue(), offset(1e-9d)); - assertThat(result2.getStandardDeviation().doubleValue()) - .isCloseTo(result1.getStandardDeviation().doubleValue(), offset(1e-12d)); - } - }.summarize(intValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java deleted file mode 100644 index 35e927c38f045..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java +++ /dev/null @@ -1,138 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link LongSummaryAggregator}. */ -class LongSummaryAggregatorTest { - - @Test - void testIsNan() { - LongSummaryAggregator ag = new LongSummaryAggregator(); - // always false for Long - assertThat(ag.isNan(-1L)).isFalse(); - assertThat(ag.isNan(0L)).isFalse(); - assertThat(ag.isNan(23L)).isFalse(); - assertThat(ag.isNan(Long.MAX_VALUE)).isFalse(); - assertThat(ag.isNan(Long.MIN_VALUE)).isFalse(); - assertThat(ag.isNan(null)).isFalse(); - } - - @Test - void testIsInfinite() { - LongSummaryAggregator ag = new LongSummaryAggregator(); - // always false for Long - assertThat(ag.isInfinite(-1L)).isFalse(); - assertThat(ag.isInfinite(0L)).isFalse(); - assertThat(ag.isInfinite(23L)).isFalse(); - assertThat(ag.isInfinite(Long.MAX_VALUE)).isFalse(); - assertThat(ag.isInfinite(Long.MIN_VALUE)).isFalse(); - assertThat(ag.isInfinite(null)).isFalse(); - } - - @Test - void testMean() { - assertThat(summarize(0L, 100L).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0L, 0L, 100L).getMean()).isCloseTo(33.333333, offset(0.00001)); - assertThat(summarize(0L, 0L, 100L, 100L).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0L, 100L, null).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize().getMean()).isNull(); - } - - @Test - void testSum() throws Exception { - assertThat(summarize(0L, 100L).getSum().longValue()).isEqualTo(100L); - assertThat(summarize(1L, 2L, 3L, 4L, 5L).getSum().longValue()).isEqualTo(15L); - assertThat(summarize(-100L, 0L, 100L, null).getSum().longValue()).isZero(); - assertThat(summarize(-10L, 100L, null).getSum().longValue()).isEqualTo(90L); - assertThat(summarize().getSum()).isNull(); - } - - @Test - void testMax() { - assertThat(summarize(-1000L, 0L, 1L, 50L, 999L, 1001L).getMax().longValue()) - .isEqualTo(1001L); - assertThat( - summarize(1L, 8L, 7L, 6L, 9L, 10L, 2L, 3L, 5L, 0L, 11L, -2L, 3L) - .getMax() - .longValue()) - .isEqualTo(11L); - assertThat( - summarize(1L, 8L, 7L, 6L, 9L, null, 10L, 2L, 3L, 5L, null, 0L, 11L, -2L, 3L) - .getMax() - .longValue()) - .isEqualTo(11L); - assertThat(summarize().getMax()).isNull(); - } - - @Test - void testMin() { - assertThat(summarize(-1000L, 0L, 1L, 50L, 999L, 1001L).getMin().longValue()) - .isEqualTo(-1000L); - assertThat( - summarize(1L, 8L, 7L, 6L, 9L, 10L, 2L, 3L, 5L, 0L, 11L, -2L, 3L) - .getMin() - .longValue()) - .isEqualTo(-2L); - assertThat( - summarize(1L, 8L, 7L, 6L, 9L, null, 10L, 2L, 3L, 5L, null, 0L, 11L, -2L, 3L) - .getMin() - .longValue()) - .isEqualTo(-2L); - assertThat(summarize().getMin()).isNull(); - } - - /** Helper method for summarizing a list of values. */ - protected NumericColumnSummary summarize(Long... values) { - return new AggregateCombineHarness< - Long, NumericColumnSummary, LongSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - - assertThat(result2.getTotalCount()).isEqualTo(result1.getTotalCount()); - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getMissingCount()).isEqualTo(result1.getMissingCount()); - assertThat(result2.getNonMissingCount()).isEqualTo(result1.getNonMissingCount()); - assertThat(result2.getInfinityCount()).isEqualTo(result1.getInfinityCount()); - assertThat(result2.getNanCount()).isEqualTo(result1.getNanCount()); - - assertThat(result2.containsNull()).isEqualTo(result1.containsNull()); - assertThat(result2.containsNonNull()).isEqualTo(result1.containsNonNull()); - - assertThat(result2.getMin().longValue()).isEqualTo(result1.getMin().longValue()); - assertThat(result2.getMax().longValue()).isEqualTo(result1.getMax().longValue()); - assertThat(result2.getSum().longValue()).isEqualTo(result1.getSum().longValue()); - assertThat(result2.getMean().doubleValue()) - .isCloseTo(result1.getMean().doubleValue(), offset(1e-12d)); - assertThat(result2.getVariance().doubleValue()) - .isCloseTo(result1.getVariance().doubleValue(), offset(1e-9d)); - assertThat(result2.getStandardDeviation().doubleValue()) - .isCloseTo(result1.getStandardDeviation().doubleValue(), offset(1e-12d)); - } - }.summarize(values); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java deleted file mode 100644 index a00564a4792d2..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java +++ /dev/null @@ -1,72 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; -import org.apache.flink.types.LongValue; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ValueSummaryAggregator.LongValueSummaryAggregator}. */ -class LongValueSummaryAggregatorTest extends LongSummaryAggregatorTest { - - /** Helper method for summarizing a list of values. */ - @Override - protected NumericColumnSummary summarize(Long... values) { - - LongValue[] longValues = new LongValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - longValues[i] = new LongValue(values[i]); - } - } - - return new AggregateCombineHarness< - LongValue, - NumericColumnSummary, - ValueSummaryAggregator.LongValueSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - - assertThat(result2.getTotalCount()).isEqualTo(result1.getTotalCount()); - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getMissingCount()).isEqualTo(result1.getMissingCount()); - assertThat(result2.getNonMissingCount()).isEqualTo(result1.getNonMissingCount()); - assertThat(result2.getInfinityCount()).isEqualTo(result1.getInfinityCount()); - assertThat(result2.getNanCount()).isEqualTo(result1.getNanCount()); - - assertThat(result2.containsNull()).isEqualTo(result1.containsNull()); - assertThat(result2.containsNonNull()).isEqualTo(result1.containsNonNull()); - - assertThat(result2.getMin().longValue()).isEqualTo(result1.getMin().longValue()); - assertThat(result2.getMax().longValue()).isEqualTo(result1.getMax().longValue()); - assertThat(result2.getSum().longValue()).isEqualTo(result1.getSum().longValue()); - assertThat(result2.getMean().doubleValue()) - .isCloseTo(result1.getMean().doubleValue(), offset(1e-12d)); - assertThat(result2.getVariance().doubleValue()) - .isCloseTo(result1.getVariance().doubleValue(), offset(1e-9d)); - assertThat(result2.getStandardDeviation().doubleValue()) - .isCloseTo(result1.getStandardDeviation().doubleValue(), offset(1e-12d)); - } - }.summarize(longValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java deleted file mode 100644 index 9204ca38e0d09..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ShortSummaryAggregator}. */ -class ShortSummaryAggregatorTest { - - @Test - void testIsNan() { - ShortSummaryAggregator ag = new ShortSummaryAggregator(); - // always false for Short - assertThat(ag.isNan((short) -1)).isFalse(); - assertThat(ag.isNan((short) 0)).isFalse(); - assertThat(ag.isNan((short) 23)).isFalse(); - assertThat(ag.isNan(Short.MAX_VALUE)).isFalse(); - assertThat(ag.isNan(Short.MIN_VALUE)).isFalse(); - assertThat(ag.isNan(null)).isFalse(); - } - - @Test - void testIsInfinite() { - ShortSummaryAggregator ag = new ShortSummaryAggregator(); - // always false for Short - assertThat(ag.isInfinite((short) -1)).isFalse(); - assertThat(ag.isInfinite((short) 0)).isFalse(); - assertThat(ag.isInfinite((short) 23)).isFalse(); - assertThat(ag.isInfinite(Short.MAX_VALUE)).isFalse(); - assertThat(ag.isInfinite(Short.MIN_VALUE)).isFalse(); - assertThat(ag.isInfinite(null)).isFalse(); - } - - @Test - void testMean() { - assertThat(summarize(0, 100).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0, 0, 100).getMean()).isCloseTo(33.333333, offset(0.00001)); - assertThat(summarize(0, 0, 100, 100).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize(0, 100, null).getMean()).isCloseTo(50.0, offset(0.0)); - assertThat(summarize().getMean()).isNull(); - } - - @Test - void testSum() throws Exception { - assertThat(summarize(0, 100).getSum().shortValue()).isEqualTo((short) 100); - assertThat(summarize(1, 2, 3, 4, 5).getSum().shortValue()).isEqualTo((short) 15); - assertThat(summarize(-100, 0, 100, null).getSum().shortValue()).isZero(); - assertThat(summarize(-10, 100, null).getSum().shortValue()).isEqualTo((short) 90); - assertThat(summarize().getSum()).isNull(); - } - - @Test - void testMax() { - assertThat(summarize(-1000, 0, 1, 50, 999, 1001).getMax().shortValue()) - .isEqualTo((short) 1001); - assertThat(summarize((int) Short.MIN_VALUE, -1000, 0).getMax().shortValue()).isZero(); - assertThat(summarize(1, 8, 7, 6, 9, 10, 2, 3, 5, 0, 11, -2, 3).getMax().shortValue()) - .isEqualTo((short) 11); - assertThat( - summarize(1, 8, 7, 6, 9, null, 10, 2, 3, 5, null, 0, 11, -2, 3) - .getMax() - .shortValue()) - .isEqualTo((short) 11); - assertThat(summarize().getMax()).isNull(); - } - - @Test - void testMin() { - assertThat(summarize(-1000, 0, 1, 50, 999, 1001).getMin().shortValue()) - .isEqualTo((short) -1000); - assertThat(summarize((int) Short.MIN_VALUE, -1000, 0).getMin().shortValue()) - .isEqualTo(Short.MIN_VALUE); - assertThat(summarize(1, 8, 7, 6, 9, 10, 2, 3, 5, 0, 11, -2, 3).getMin().shortValue()) - .isEqualTo((short) -2); - assertThat( - summarize(1, 8, 7, 6, 9, null, 10, 2, 3, 5, null, 0, 11, -2, 3) - .getMin() - .shortValue()) - .isEqualTo((short) -2); - assertThat(summarize().getMin()).isNull(); - } - - /** Helper method for summarizing a list of values. */ - protected NumericColumnSummary summarize(Integer... values) { - - // cast everything to short here - Short[] shortValues = new Short[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - shortValues[i] = values[i].shortValue(); - } - } - - return new AggregateCombineHarness< - Short, NumericColumnSummary, ShortSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - - assertThat(result2.getTotalCount()).isEqualTo(result1.getTotalCount()); - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getMissingCount()).isEqualTo(result1.getMissingCount()); - assertThat(result2.getNonMissingCount()).isEqualTo(result1.getNonMissingCount()); - assertThat(result2.getInfinityCount()).isEqualTo(result1.getInfinityCount()); - assertThat(result2.getNanCount()).isEqualTo(result1.getNanCount()); - - assertThat(result2.containsNull()).isEqualTo(result1.containsNull()); - assertThat(result2.containsNonNull()).isEqualTo(result1.containsNonNull()); - - assertThat(result2.getMin().shortValue()).isEqualTo(result1.getMin().shortValue()); - assertThat(result2.getMax().shortValue()).isEqualTo(result1.getMax().shortValue()); - assertThat(result2.getSum().shortValue()).isEqualTo(result1.getSum().shortValue()); - assertThat(result2.getMean().doubleValue()) - .isCloseTo(result1.getMean().doubleValue(), offset(1e-12d)); - assertThat(result2.getVariance().doubleValue()) - .isCloseTo(result1.getVariance().doubleValue(), offset(1e-9d)); - assertThat(result2.getStandardDeviation().doubleValue()) - .isCloseTo(result1.getStandardDeviation().doubleValue(), offset(1e-12d)); - } - }.summarize(shortValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java deleted file mode 100644 index c109fc2bb411a..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java +++ /dev/null @@ -1,71 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.NumericColumnSummary; -import org.apache.flink.types.ShortValue; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ValueSummaryAggregator.ShortValueSummaryAggregator}. */ -class ShortValueSummaryAggregatorTest extends ShortSummaryAggregatorTest { - - /** Helper method for summarizing a list of values. */ - protected NumericColumnSummary summarize(Integer... values) { - - ShortValue[] shortValues = new ShortValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - shortValues[i] = new ShortValue(values[i].shortValue()); - } - } - - return new AggregateCombineHarness< - ShortValue, - NumericColumnSummary, - ValueSummaryAggregator.ShortValueSummaryAggregator>() { - - @Override - protected void compareResults( - NumericColumnSummary result1, NumericColumnSummary result2) { - - assertThat(result2.getTotalCount()).isEqualTo(result1.getTotalCount()); - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getMissingCount()).isEqualTo(result1.getMissingCount()); - assertThat(result2.getNonMissingCount()).isEqualTo(result1.getNonMissingCount()); - assertThat(result2.getInfinityCount()).isEqualTo(result1.getInfinityCount()); - assertThat(result2.getNanCount()).isEqualTo(result1.getNanCount()); - - assertThat(result2.containsNull()).isEqualTo(result1.containsNull()); - assertThat(result2.containsNonNull()).isEqualTo(result1.containsNonNull()); - - assertThat(result2.getMin().shortValue()).isEqualTo(result1.getMin().shortValue()); - assertThat(result2.getMax().shortValue()).isEqualTo(result1.getMax().shortValue()); - assertThat(result2.getSum().shortValue()).isEqualTo(result1.getSum().shortValue()); - assertThat(result2.getMean().doubleValue()) - .isCloseTo(result1.getMean().doubleValue(), offset(1e-12d)); - assertThat(result2.getVariance().doubleValue()) - .isCloseTo(result1.getVariance().doubleValue(), offset(1e-9d)); - assertThat(result2.getStandardDeviation().doubleValue()) - .isCloseTo(result1.getStandardDeviation().doubleValue(), offset(1e-12d)); - } - }.summarize(shortValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java deleted file mode 100644 index a3b8a393cc35a..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java +++ /dev/null @@ -1,95 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.StringColumnSummary; - -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link StringSummaryAggregator}. */ -class StringSummaryAggregatorTest { - - @Test - void testMixedGroup() { - StringColumnSummary summary = - summarize("abc", "", null, " ", "defghi", "foo", null, null, "", " "); - assertThat(summary.getTotalCount()).isEqualTo(10); - assertThat(summary.getNullCount()).isEqualTo(3); - assertThat(summary.getNonNullCount()).isEqualTo(7); - assertThat(summary.getEmptyCount()).isEqualTo(2); - assertThat(summary.getMinLength().intValue()).isZero(); - assertThat(summary.getMaxLength().intValue()).isEqualTo(6); - assertThat(summary.getMeanLength().doubleValue()).isCloseTo(2.142857, offset(0.001)); - } - - @Test - void testAllNullStrings() { - StringColumnSummary summary = summarize(null, null, null, null); - assertThat(summary.getTotalCount()).isEqualTo(4); - assertThat(summary.getNullCount()).isEqualTo(4); - assertThat(summary.getNonNullCount()).isZero(); - assertThat(summary.getEmptyCount()).isZero(); - assertThat(summary.getMinLength()).isNull(); - assertThat(summary.getMaxLength()).isNull(); - assertThat(summary.getMeanLength()).isNull(); - } - - @Test - void testAllWithValues() { - StringColumnSummary summary = summarize("cat", "hat", "dog", "frog"); - assertThat(summary.getTotalCount()).isEqualTo(4); - assertThat(summary.getNullCount()).isZero(); - assertThat(summary.getNonNullCount()).isEqualTo(4); - assertThat(summary.getEmptyCount()).isZero(); - assertThat(summary.getMinLength().intValue()).isEqualTo(3); - assertThat(summary.getMaxLength().intValue()).isEqualTo(4); - assertThat(summary.getMeanLength().doubleValue()).isCloseTo(3.25, offset(0.0)); - } - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - protected StringColumnSummary summarize(String... values) { - - return new AggregateCombineHarness() { - - @Override - protected void compareResults( - StringColumnSummary result1, StringColumnSummary result2) { - assertThat(result2.getEmptyCount()).isEqualTo(result1.getEmptyCount()); - assertThat(result2.getMaxLength()).isEqualTo(result1.getMaxLength()); - assertThat(result2.getMinLength()).isEqualTo(result1.getMinLength()); - if (result1.getMeanLength() == null) { - assertThat(result2.getMeanLength()).isEqualTo(result1.getMeanLength()); - } else { - assertThat(result2.getMeanLength().doubleValue()) - .isCloseTo(result1.getMeanLength().doubleValue(), offset(1e-5d)); - } - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getNonNullCount()).isEqualTo(result1.getNonNullCount()); - } - }.summarize(values); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java deleted file mode 100644 index 248f4a2f5bbbd..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java +++ /dev/null @@ -1,69 +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.api.java.summarize.aggregation; - -import org.apache.flink.api.java.summarize.StringColumnSummary; -import org.apache.flink.types.StringValue; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; - -/** Tests for {@link ValueSummaryAggregator.StringValueSummaryAggregator}. */ -class StringValueSummaryAggregatorTest extends StringSummaryAggregatorTest { - - /** - * Helper method for summarizing a list of values. - * - *

This method breaks the rule of "testing only one thing" by aggregating and combining a - * bunch of different ways. - */ - @Override - protected StringColumnSummary summarize(String... values) { - - StringValue[] stringValues = new StringValue[values.length]; - for (int i = 0; i < values.length; i++) { - if (values[i] != null) { - stringValues[i] = new StringValue(values[i]); - } - } - - return new AggregateCombineHarness< - StringValue, - StringColumnSummary, - ValueSummaryAggregator.StringValueSummaryAggregator>() { - - @Override - protected void compareResults( - StringColumnSummary result1, StringColumnSummary result2) { - assertThat(result2.getEmptyCount()).isEqualTo(result1.getEmptyCount()); - assertThat(result2.getMaxLength()).isEqualTo(result1.getMaxLength()); - assertThat(result2.getMinLength()).isEqualTo(result1.getMinLength()); - if (result1.getMeanLength() == null) { - assertThat(result2.getMeanLength()).isEqualTo(result1.getMeanLength()); - } else { - assertThat(result2.getMeanLength().doubleValue()) - .isCloseTo(result1.getMeanLength().doubleValue(), offset(1e-5d)); - } - - assertThat(result2.getNullCount()).isEqualTo(result1.getNullCount()); - assertThat(result2.getNonNullCount()).isEqualTo(result1.getNonNullCount()); - } - }.summarize(stringValues); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java deleted file mode 100644 index 735fb0af0dac0..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java +++ /dev/null @@ -1,78 +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.api.java.summarize.aggregation; - -import org.apache.flink.types.BooleanValue; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.ShortValue; -import org.apache.flink.types.StringValue; - -import org.junit.jupiter.api.Test; - -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link SummaryAggregatorFactory}. */ -class SummaryAggregatorFactoryTest { - - @Test - void testCreate() { - // supported primitive types - assertThat(SummaryAggregatorFactory.create(String.class).getClass()) - .isEqualTo(StringSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(Short.class).getClass()) - .isEqualTo(ShortSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(Integer.class).getClass()) - .isEqualTo(IntegerSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(Long.class).getClass()) - .isEqualTo(LongSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(Float.class).getClass()) - .isEqualTo(FloatSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(Double.class).getClass()) - .isEqualTo(DoubleSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(Boolean.class).getClass()) - .isEqualTo(BooleanSummaryAggregator.class); - - // supported value types - assertThat(SummaryAggregatorFactory.create(StringValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.StringValueSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(ShortValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.ShortValueSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(IntValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.IntegerValueSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(LongValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.LongValueSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(FloatValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.FloatValueSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(DoubleValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.DoubleValueSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(BooleanValue.class).getClass()) - .isEqualTo(ValueSummaryAggregator.BooleanValueSummaryAggregator.class); - - // some not well supported types - these fallback to ObjectSummaryAggregator - assertThat(SummaryAggregatorFactory.create(Object.class).getClass()) - .isEqualTo(ObjectSummaryAggregator.class); - assertThat(SummaryAggregatorFactory.create(List.class).getClass()) - .isEqualTo(ObjectSummaryAggregator.class); - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java b/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java deleted file mode 100644 index 59ffaf182746c..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java +++ /dev/null @@ -1,536 +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.api.java.tuple; - -import org.apache.flink.util.FileUtils; - -import java.io.File; -import java.io.IOException; -import java.util.Scanner; - -/** - * Source code generator for tuple classes and classes which depend on the arity of tuples. - * - *

This class is responsible for generating tuple-size dependent code in the {@link - * org.apache.flink.api.java.io.CsvReader}, {@link - * org.apache.flink.api.java.operators.ProjectOperator}, {@link - * org.apache.flink.api.java.operators.JoinOperator.JoinProjection} and {@link - * org.apache.flink.api.java.operators.CrossOperator.CrossProjection}. - */ -class TupleGenerator { - - // Parameters for tuple classes - - private static final String ROOT_DIRECTORY = "./flink-java/src/main/java"; - - private static final String GEN_TYPE_PREFIX = "T"; - - // Parameters for tuple-dependent classes - private static final String BEGIN_INDICATOR = "BEGIN_OF_TUPLE_DEPENDENT_CODE"; - - private static final String END_INDICATOR = "END_OF_TUPLE_DEPENDENT_CODE"; - - // Parameters for CsvReader - private static final String CSV_READER_PACKAGE = "org.apache.flink.api.java.io"; - - private static final String CSV_READER_CLASSNAME = "CsvReader"; - - // Parameters for ProjectOperator - private static final String PROJECT_OPERATOR_PACKAGE = "org.apache.flink.api.java.operators"; - - private static final String PROJECT_OPERATOR_CLASSNAME = "ProjectOperator"; - - // Parameters for JoinOperator - private static final String JOIN_OPERATOR_PACKAGE = "org.apache.flink.api.java.operators"; - - private static final String JOIN_OPERATOR_CLASSNAME = "JoinOperator"; - - // parameters for CrossOperator - private static final String CROSS_OPERATOR_PACKAGE = "org.apache.flink.api.java.operators"; - - private static final String CROSS_OPERATOR_CLASSNAME = "CrossOperator"; - - // min. and max. tuple arity - private static final int FIRST = 1; - - private static final int LAST = 25; - - public static void main(String[] args) throws Exception { - System.err.println("Current directory " + System.getProperty("user.dir")); - String rootDir = ROOT_DIRECTORY; - if (args.length > 0) { - rootDir = args[0] + "/" + ROOT_DIRECTORY; - } - System.err.println("Using root directory: " + rootDir); - File root = new File(rootDir); - - modifyCsvReader(root); - - modifyProjectOperator(root); - - modifyJoinProjectOperator(root); - - modifyCrossProjectOperator(root); - } - - private static File getPackage(File root, String packageString) { - File dir = new File(root, packageString.replace('.', '/')); - if (!dir.exists() && dir.isDirectory()) { - System.err.println("None existent directory: " + dir.getAbsolutePath()); - System.exit(1); - } - return dir; - } - - private static void insertCodeIntoFile(String code, File file) throws IOException { - String fileContent = FileUtils.readFileUtf8(file); - - try (Scanner s = new Scanner(fileContent)) { - StringBuilder sb = new StringBuilder(); - String line; - - boolean indicatorFound = false; - - // add file beginning - while (s.hasNextLine() && (line = s.nextLine()) != null) { - sb.append(line).append("\n"); - if (line.contains(BEGIN_INDICATOR)) { - indicatorFound = true; - break; - } - } - - if (!indicatorFound) { - System.out.println( - "No indicator found in '" + file + "'. Will skip code generation."); - s.close(); - return; - } - - // add generator signature - sb.append("\t// GENERATED FROM ").append(TupleGenerator.class.getName()).append(".\n"); - - // add tuple dependent code - sb.append(code).append("\n"); - - // skip generated code - while (s.hasNextLine() && (line = s.nextLine()) != null) { - if (line.contains(END_INDICATOR)) { - sb.append(line).append("\n"); - break; - } - } - - // add file ending - while (s.hasNextLine() && (line = s.nextLine()) != null) { - sb.append(line).append("\n"); - } - s.close(); - FileUtils.writeFileUtf8(file, sb.toString()); - } - } - - private static void modifyCrossProjectOperator(File root) throws IOException { - // generate code - StringBuilder sb = new StringBuilder(); - - // method begin - sb.append("\n"); - - // method comment - sb.append("\t\t/**\n"); - sb.append("\t\t * Chooses a projectTupleX according to the length of\n"); - sb.append( - "\t\t * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#fieldIndexes} \n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @return The projected DataSet.\n"); - sb.append("\t\t */\n"); - - // method signature - sb.append("\t\t@SuppressWarnings(\"unchecked\")\n"); - sb.append("\t\tpublic ProjectCross projectTupleX() {\n"); - sb.append("\t\t\tProjectCross projectionCross = null;\n\n"); - sb.append("\t\t\tswitch (fieldIndexes.length) {\n"); - for (int numFields = FIRST; numFields <= LAST; numFields++) { - sb.append( - "\t\t\tcase " - + numFields - + ":" - + " projectionCross = (ProjectCross) projectTuple" - + numFields - + "(); break;\n"); - } - sb.append( - "\t\t\tdefault: throw new IllegalStateException(\"Excessive arity in tuple.\");\n"); - sb.append("\t\t\t}\n\n"); - sb.append("\t\t\treturn projectionCross;\n"); - - // method end - sb.append("\t\t}\n"); - - for (int numFields = FIRST; numFields <= LAST; numFields++) { - - // method begin - sb.append("\n"); - - // method comment - sb.append("\t\t/**\n"); - sb.append( - "\t\t * Projects a pair of crossed elements to a {@link Tuple} with the previously selected fields. \n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @return The projected data set.\n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @see Tuple\n"); - sb.append("\t\t * @see DataSet\n"); - sb.append("\t\t */\n"); - - // method signature - sb.append("\t\tpublic <"); - appendTupleTypeGenerics(sb, numFields); - sb.append("> ProjectCross> projectTuple" + numFields + "("); - sb.append(") {\n"); - - // extract field types - sb.append("\t\t\tTypeInformation[] fTypes = extractFieldTypes(fieldIndexes);\n"); - - // create new tuple type info - sb.append("\t\t\tTupleTypeInfo> tType = new TupleTypeInfo>(fTypes);\n\n"); - - // create and return new project operator - sb.append("\t\t\treturn new ProjectCross>(this.ds1, this.ds2, this.fieldIndexes, this.isFieldInFirst, tType, this, hint);\n"); - - // method end - sb.append("\t\t}\n"); - } - - // insert code into file - File dir = getPackage(root, CROSS_OPERATOR_PACKAGE); - File projectOperatorClass = new File(dir, CROSS_OPERATOR_CLASSNAME + ".java"); - insertCodeIntoFile(sb.toString(), projectOperatorClass); - } - - private static void modifyProjectOperator(File root) throws IOException { - // generate code - StringBuilder sb = new StringBuilder(); - - // method begin - sb.append("\n"); - - // method comment - sb.append("\t\t/**\n"); - sb.append("\t\t * Chooses a projectTupleX according to the length of\n"); - sb.append( - "\t\t * {@link org.apache.flink.api.java.operators.ProjectOperator.Projection#fieldIndexes} \n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @return The projected DataSet.\n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @see org.apache.flink.api.java.operators.ProjectOperator.Projection\n"); - sb.append("\t\t */\n"); - - // method signature - sb.append("\t\t@SuppressWarnings(\"unchecked\")\n"); - sb.append("\t\tpublic ProjectOperator projectTupleX() {\n"); - sb.append("\t\t\tProjectOperator projOperator;\n\n"); - sb.append("\t\t\tswitch (fieldIndexes.length) {\n"); - for (int numFields = FIRST; numFields <= LAST; numFields++) { - sb.append( - "\t\t\tcase " - + numFields - + ":" - + " projOperator = (ProjectOperator) projectTuple" - + numFields - + "(); break;\n"); - } - sb.append( - "\t\t\tdefault: throw new IllegalStateException(\"Excessive arity in tuple.\");\n"); - sb.append("\t\t\t}\n\n"); - sb.append("\t\t\treturn projOperator;\n"); - - // method end - sb.append("\t\t}\n"); - - for (int numFields = FIRST; numFields <= LAST; numFields++) { - - // method begin - sb.append("\n"); - - // method comment - sb.append("\t\t/**\n"); - sb.append( - "\t\t * Projects a {@link Tuple} {@link DataSet} to the previously selected fields. \n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @return The projected DataSet.\n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @see Tuple\n"); - sb.append("\t\t * @see DataSet\n"); - sb.append("\t\t */\n"); - - // method signature - sb.append("\t\tpublic <"); - appendTupleTypeGenerics(sb, numFields); - sb.append("> ProjectOperator> projectTuple" + numFields + "("); - sb.append(") {\n"); - - // extract field types - sb.append( - "\t\t\tTypeInformation[] fTypes = extractFieldTypes(fieldIndexes, ds.getType());\n"); - - // create new tuple type info - sb.append("\t\t\tTupleTypeInfo> tType = new TupleTypeInfo>(fTypes);\n\n"); - - // create and return new project operator - sb.append("\t\t\treturn new ProjectOperator>(this.ds, this.fieldIndexes, tType);\n"); - - // method end - sb.append("\t\t}\n"); - } - - // insert code into file - File dir = getPackage(root, PROJECT_OPERATOR_PACKAGE); - File projectOperatorClass = new File(dir, PROJECT_OPERATOR_CLASSNAME + ".java"); - insertCodeIntoFile(sb.toString(), projectOperatorClass); - } - - private static void modifyJoinProjectOperator(File root) throws IOException { - // generate code - StringBuilder sb = new StringBuilder(); - - // method begin - sb.append("\n"); - - // method comment - sb.append("\t\t/**\n"); - sb.append("\t\t * Chooses a projectTupleX according to the length of\n"); - sb.append( - "\t\t * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#fieldIndexes}\n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @return The projected DataSet.\n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @see org.apache.flink.api.java.operators.JoinOperator.ProjectJoin\n"); - sb.append("\t\t */\n"); - - // method signature - sb.append("\t\t@SuppressWarnings(\"unchecked\")\n"); - sb.append("\t\tpublic ProjectJoin projectTupleX() {\n"); - sb.append("\t\t\tProjectJoin projectJoin = null;\n\n"); - sb.append("\t\t\tswitch (fieldIndexes.length) {\n"); - for (int numFields = FIRST; numFields <= LAST; numFields++) { - sb.append( - "\t\t\tcase " - + numFields - + ":" - + " projectJoin = (ProjectJoin) projectTuple" - + numFields - + "(); break;\n"); - } - sb.append( - "\t\t\tdefault: throw new IllegalStateException(\"Excessive arity in tuple.\");\n"); - sb.append("\t\t\t}\n\n"); - sb.append("\t\t\treturn projectJoin;\n"); - - // method end - sb.append("\t\t}\n"); - - for (int numFields = FIRST; numFields <= LAST; numFields++) { - - // method begin - sb.append("\n"); - - // method comment - sb.append("\t\t/**\n"); - sb.append( - "\t\t * Projects a pair of joined elements to a {@link Tuple} with the previously selected fields. \n"); - sb.append("\t\t * Requires the classes of the fields of the resulting tuples. \n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @return The projected data set.\n"); - sb.append("\t\t * \n"); - sb.append("\t\t * @see Tuple\n"); - sb.append("\t\t * @see DataSet\n"); - sb.append("\t\t */\n"); - - // method signature - sb.append("\t\tpublic <"); - appendTupleTypeGenerics(sb, numFields); - sb.append("> ProjectJoin> projectTuple" + numFields + "("); - sb.append(") {\n"); - - // extract field types - sb.append("\t\t\tTypeInformation[] fTypes = extractFieldTypes(fieldIndexes);\n"); - - // create new tuple type info - sb.append("\t\t\tTupleTypeInfo> tType = new TupleTypeInfo>(fTypes);\n\n"); - - // create and return new project operator - sb.append("\t\t\treturn new ProjectJoin>(this.ds1, this.ds2, this.keys1, this.keys2, this.hint, this.fieldIndexes, this.isFieldInFirst, tType, this);\n"); - - // method end - sb.append("\t\t}\n"); - } - - // insert code into file - File dir = getPackage(root, JOIN_OPERATOR_PACKAGE); - File projectOperatorClass = new File(dir, JOIN_OPERATOR_CLASSNAME + ".java"); - insertCodeIntoFile(sb.toString(), projectOperatorClass); - } - - private static void modifyCsvReader(File root) throws IOException { - // generate code - StringBuilder sb = new StringBuilder(1000); - for (int numFields = FIRST; numFields <= LAST; numFields++) { - - // method begin - sb.append("\n"); - - // java doc - sb.append("\t/**\n"); - sb.append( - "\t * Specifies the types for the CSV fields. This method parses the CSV data to a ") - .append(numFields) - .append("-tuple\n"); - sb.append("\t * which has fields of the specified types.\n"); - sb.append( - "\t * This method is overloaded for each possible length of the tuples to support type safe\n"); - sb.append("\t * creation of data sets through CSV parsing.\n"); - sb.append("\t *\n"); - - for (int pos = 0; pos < numFields; pos++) { - sb.append("\t * @param type").append(pos); - sb.append(" The type of CSV field ").append(pos).append(" and the type of field "); - sb.append(pos).append(" in the returned tuple type.\n"); - } - sb.append( - "\t * @return The {@link org.apache.flink.api.java.DataSet} representing the parsed CSV data.\n"); - sb.append("\t */\n"); - - // method signature - sb.append("\tpublic <"); - appendTupleTypeGenerics(sb, numFields); - sb.append("> DataSource> types("); - for (int i = 0; i < numFields; i++) { - if (i > 0) { - sb.append(", "); - } - sb.append("Class<"); - sb.append(GEN_TYPE_PREFIX + i); - sb.append("> type" + i); - } - sb.append(") {\n"); - - // get TupleTypeInfo - sb.append("\t\tTupleTypeInfo> types = TupleTypeInfo.getBasicAndBasicValueTupleTypeInfo("); - for (int i = 0; i < numFields; i++) { - if (i > 0) { - sb.append(", "); - } - sb.append("type" + i); - } - sb.append(");\n"); - - // create csv input format - sb.append("\t\tCsvInputFormat> inputFormat = new TupleCsvInputFormat>(path, types, this.includedMask);\n"); - - // configure input format - sb.append("\t\tconfigureInputFormat(inputFormat);\n"); - - // return - sb.append("\t\treturn new DataSource>(executionContext, inputFormat, types, Utils.getCallLocationName());\n"); - - // end of method - sb.append("\t}\n"); - } - - // insert code into file - File dir = getPackage(root, CSV_READER_PACKAGE); - File csvReaderClass = new File(dir, CSV_READER_CLASSNAME + ".java"); - insertCodeIntoFile(sb.toString(), csvReaderClass); - } - - private static void appendTupleTypeGenerics(StringBuilder sb, int numFields) { - for (int i = 0; i < numFields; i++) { - if (i > 0) { - sb.append(", "); - } - sb.append(GEN_TYPE_PREFIX + i); - } - } - - private static final String HEADER = - "/*\n" - + " * Licensed to the Apache Software Foundation (ASF) under one\n" - + " * or more contributor license agreements. See the NOTICE file\n" - + " * distributed with this work for additional information\n" - + " * regarding copyright ownership. The ASF licenses this file\n" - + " * to you under the Apache License, Version 2.0 (the\n" - + " * \"License\"); you may not use this file except in compliance\n" - + " * with the License. You may obtain a copy of the License at\n" - + " *\n" - + " * http://www.apache.org/licenses/LICENSE-2.0\n" - + " *\n" - + " * Unless required by applicable law or agreed to in writing, software\n" - + " * distributed under the License is distributed on an \"AS IS\" BASIS,\n" - + " * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" - + " * See the License for the specific language governing permissions and\n" - + " * limitations under the License.\n" - + " */" - + "\n" - + "\n" - + "\n" - + "// --------------------------------------------------------------\n" - + "// THIS IS A GENERATED SOURCE FILE. DO NOT EDIT!\n" - + "// GENERATED FROM " - + TupleGenerator.class.getName() - + ".\n" - + "// --------------------------------------------------------------\n\n\n"; -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/PlanGeneratorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/PlanGeneratorTest.java deleted file mode 100644 index eaf8d0ab21293..0000000000000 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/PlanGeneratorTest.java +++ /dev/null @@ -1,79 +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.api.java.utils; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; - -import org.junit.jupiter.api.Test; - -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link PlanGenerator}. */ -class PlanGeneratorTest { - - @Test - void testGenerate() { - - final String fileA = "fileA"; - final String fileB = "fileB"; - - final Map originalArtifacts = - Stream.of( - Tuple2.of( - fileA, - new DistributedCache.DistributedCacheEntry("test1", true)), - Tuple2.of( - fileB, - new DistributedCache.DistributedCacheEntry("test2", false))) - .collect(Collectors.toMap(x -> x.f0, x -> x.f1)); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(10); - env.registerCachedFile("test1", fileA, true); - env.registerCachedFile("test2", fileB, false); - - env.fromElements(1, 3, 5) - .map((MapFunction) value -> String.valueOf(value + 1)) - .writeAsText("/tmp/csv"); - - final Plan generatedPlanUnderTest = env.createProgramPlan("test"); - - final Map retrievedArtifacts = - generatedPlanUnderTest.getCachedFiles().stream() - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - assertThat(generatedPlanUnderTest.getDataSinks()).hasSize(1); - assertThat(generatedPlanUnderTest.getDefaultParallelism()).isEqualTo(10); - assertThat(generatedPlanUnderTest.getExecutionConfig()).isEqualTo(env.getConfig()); - assertThat(generatedPlanUnderTest.getJobName()).isEqualTo("test"); - - assertThat(retrievedArtifacts) - .hasSameSizeAs(originalArtifacts) - .containsEntry(fileA, originalArtifacts.get(fileA)) - .containsEntry(fileB, originalArtifacts.get(fileB)); - } -} diff --git a/flink-java/src/test/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory b/flink-java/src/test/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory deleted file mode 100644 index c09254a496b30..0000000000000 --- a/flink-java/src/test/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory +++ /dev/null @@ -1,16 +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. - -org.apache.flink.api.java.ExecutorDiscoveryAndJobClientTest$IDReportingExecutorFactory \ No newline at end of file diff --git a/flink-java/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/flink-java/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension deleted file mode 100644 index 0b74fd4603ce2..0000000000000 --- a/flink-java/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension +++ /dev/null @@ -1,16 +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. - -org.apache.flink.util.TestLoggerExtension diff --git a/flink-java/src/test/resources/log4j2-test.properties b/flink-java/src/test/resources/log4j2-test.properties deleted file mode 100644 index 835c2ec9a3d02..0000000000000 --- a/flink-java/src/test/resources/log4j2-test.properties +++ /dev/null @@ -1,28 +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. -################################################################################ - -# Set root logger level to OFF to not flood build logs -# set manually to INFO for debugging purposes -rootLogger.level = OFF -rootLogger.appenderRef.test.ref = TestLogger - -appender.testlogger.name = TestLogger -appender.testlogger.type = CONSOLE -appender.testlogger.target = SYSTEM_ERR -appender.testlogger.layout.type = PatternLayout -appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-libraries/flink-state-processing-api/pom.xml b/flink-libraries/flink-state-processing-api/pom.xml index a455936084af8..a32253408b0bf 100644 --- a/flink-libraries/flink-state-processing-api/pom.xml +++ b/flink-libraries/flink-state-processing-api/pom.xml @@ -53,12 +53,6 @@ under the License. ${project.version} provided - - org.apache.flink - flink-java - ${project.version} - provided - diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/SavepointReader.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/SavepointReader.java index 69761195a5216..ec4850e8f24d5 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/SavepointReader.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/SavepointReader.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -48,6 +47,7 @@ import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import javax.annotation.Nullable; diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml deleted file mode 100644 index b32051bed90ea..0000000000000 --- a/flink-optimizer/pom.xml +++ /dev/null @@ -1,109 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-parent - 2.0-SNAPSHOT - - - flink-optimizer - Flink : Optimizer - - jar - - - - - - - org.apache.flink - flink-core - ${project.version} - - - - org.apache.flink - flink-runtime - ${project.version} - - - - org.apache.flink - flink-java - ${project.version} - - - - org.apache.flink - flink-shaded-guava - - - - org.apache.flink - flink-shaded-jackson - - - - - - org.apache.flink - flink-test-utils-junit - - - - org.apache.flink - flink-runtime - ${project.version} - test-jar - test - - - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - - /tools/maven/suppressions-optimizer.xml - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - - diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/CompilerException.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/CompilerException.java deleted file mode 100644 index 1027b63873459..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/CompilerException.java +++ /dev/null @@ -1,56 +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.optimizer; - -/** An exception that is thrown by the Optimizer when encountering an illegal condition. */ -public class CompilerException extends RuntimeException { - - private static final long serialVersionUID = 3810067304570563755L; - - /** Creates a compiler exception with no message and no cause. */ - public CompilerException() {} - - /** - * Creates a compiler exception with the given message and no cause. - * - * @param message The message for the exception. - */ - public CompilerException(String message) { - super(message); - } - - /** - * Creates a compiler exception with the given cause and no message. - * - * @param cause The Throwable that caused this exception. - */ - public CompilerException(Throwable cause) { - super(cause); - } - - /** - * Creates a compiler exception with the given message and cause. - * - * @param message The message for the exception. - * @param cause The Throwable that caused this exception. - */ - public CompilerException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/CompilerPostPassException.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/CompilerPostPassException.java deleted file mode 100644 index d8b062b72de12..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/CompilerPostPassException.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.optimizer; - -/** - * An exception that is thrown by the Optimizer when encountering a problem during the optimizer - * post pass. This is a dedicated exception because it is thrown by user-specified optimizer - * extensions. - */ -public class CompilerPostPassException extends CompilerException { - - private static final long serialVersionUID = -322650826288034623L; - - /** Creates a post pass exception with no message and no cause. */ - public CompilerPostPassException() {} - - /** - * Creates a post pass exception with the given message and no cause. - * - * @param message The message for the exception. - */ - public CompilerPostPassException(String message) { - super(message); - } - - /** - * Creates a post pass exception with the given cause and no message. - * - * @param cause The Throwable that caused this exception. - */ - public CompilerPostPassException(Throwable cause) { - super(cause); - } - - /** - * Creates a post pass exception with the given message and cause. - * - * @param message The message for the exception. - * @param cause The Throwable that caused this exception. - */ - public CompilerPostPassException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/DataStatistics.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/DataStatistics.java deleted file mode 100644 index 856bece3fd747..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/DataStatistics.java +++ /dev/null @@ -1,67 +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.optimizer; - -import org.apache.flink.api.common.io.statistics.BaseStatistics; - -import java.util.HashMap; -import java.util.Map; - -/** - * The collection of access methods that can be used to retrieve statistical information about the - * data processed in a job. Currently this method acts as an entry point only for obtaining cached - * statistics. - */ -public class DataStatistics { - - private final Map baseStatisticsCache; - - // -------------------------------------------------------------------------------------------- - - /** Creates a new statistics object, with an empty cache. */ - public DataStatistics() { - this.baseStatisticsCache = new HashMap(); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the base statistics for the input identified by the given identifier. - * - * @param inputIdentifier The identifier for the input. - * @return The statistics that were cached for this input. - */ - public BaseStatistics getBaseStatistics(String inputIdentifier) { - synchronized (this.baseStatisticsCache) { - return this.baseStatisticsCache.get(inputIdentifier); - } - } - - /** - * Caches the given statistics. They are later retrievable under the given identifier. - * - * @param statistics The statistics to cache. - * @param identifier The identifier which may be later used to retrieve the statistics. - */ - public void cacheBaseStatistics(BaseStatistics statistics, String identifier) { - synchronized (this.baseStatisticsCache) { - this.baseStatisticsCache.put(identifier, statistics); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java deleted file mode 100644 index 5d9927fcc237c..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java +++ /dev/null @@ -1,595 +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.optimizer; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.Plan; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.costs.DefaultCostEstimator; -import org.apache.flink.optimizer.dag.DataSinkNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.SinkJoiner; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SinkJoinerPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.postpass.OptimizerPostPass; -import org.apache.flink.optimizer.traversals.BinaryUnionReplacer; -import org.apache.flink.optimizer.traversals.BranchesVisitor; -import org.apache.flink.optimizer.traversals.GraphCreatingVisitor; -import org.apache.flink.optimizer.traversals.IdAndEstimatesVisitor; -import org.apache.flink.optimizer.traversals.InterestingPropertyVisitor; -import org.apache.flink.optimizer.traversals.PlanFinalizer; -import org.apache.flink.optimizer.traversals.RangePartitionRewriter; -import org.apache.flink.optimizer.traversals.UnionParallelismAndForwardEnforcer; -import org.apache.flink.util.InstantiationUtil; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -/** - * The optimizer that takes the user specified program plan and creates an optimized plan that - * contains exact descriptions about how the physical execution will take place. It first translates - * the user program into an internal optimizer representation and then chooses between different - * alternatives for shipping strategies and local strategies. - * - *

The basic principle is taken from optimizer works in systems such as Volcano/Cascades and - * Selinger/System-R/DB2. The optimizer walks from the sinks down, generating interesting - * properties, and ascends from the sources generating alternative plans, pruning against the - * interesting properties. - * - *

The optimizer also assigns the memory to the individual tasks. This is currently done in a - * very simple fashion: All sub-tasks that need memory (e.g. reduce or join) are given an equal - * share of memory. - */ -public class Optimizer { - - // ------------------------------------------------------------------------ - // Constants - // ------------------------------------------------------------------------ - - /** - * Compiler hint key for the input channel's shipping strategy. This String is a key to the - * operator's stub parameters. The corresponding value tells the compiler which shipping - * strategy to use for the input channel. If the operator has two input channels, the shipping - * strategy is applied to both input channels. - */ - public static final String HINT_SHIP_STRATEGY = "INPUT_SHIP_STRATEGY"; - - /** - * Compiler hint key for the first input channel's shipping strategy. This String is a - * key to the operator's stub parameters. The corresponding value tells the compiler which - * shipping strategy to use for the first input channel. Only applicable to operators - * with two inputs. - */ - public static final String HINT_SHIP_STRATEGY_FIRST_INPUT = "INPUT_LEFT_SHIP_STRATEGY"; - - /** - * Compiler hint key for the second input channel's shipping strategy. This String is a - * key to the operator's stub parameters. The corresponding value tells the compiler which - * shipping strategy to use for the second input channel. Only applicable to operators - * with two inputs. - */ - public static final String HINT_SHIP_STRATEGY_SECOND_INPUT = "INPUT_RIGHT_SHIP_STRATEGY"; - - /** - * Value for the shipping strategy compiler hint that enforces a Forward strategy on the - * input channel, i.e. no redistribution of any kind. - * - * @see #HINT_SHIP_STRATEGY - * @see #HINT_SHIP_STRATEGY_FIRST_INPUT - * @see #HINT_SHIP_STRATEGY_SECOND_INPUT - */ - public static final String HINT_SHIP_STRATEGY_FORWARD = "SHIP_FORWARD"; - - /** - * Value for the shipping strategy compiler hint that enforces a random repartition strategy. - * - * @see #HINT_SHIP_STRATEGY - * @see #HINT_SHIP_STRATEGY_FIRST_INPUT - * @see #HINT_SHIP_STRATEGY_SECOND_INPUT - */ - public static final String HINT_SHIP_STRATEGY_REPARTITION = "SHIP_REPARTITION"; - - /** - * Value for the shipping strategy compiler hint that enforces a hash-partition strategy. - * - * @see #HINT_SHIP_STRATEGY - * @see #HINT_SHIP_STRATEGY_FIRST_INPUT - * @see #HINT_SHIP_STRATEGY_SECOND_INPUT - */ - public static final String HINT_SHIP_STRATEGY_REPARTITION_HASH = "SHIP_REPARTITION_HASH"; - - /** - * Value for the shipping strategy compiler hint that enforces a range-partition strategy. - * - * @see #HINT_SHIP_STRATEGY - * @see #HINT_SHIP_STRATEGY_FIRST_INPUT - * @see #HINT_SHIP_STRATEGY_SECOND_INPUT - */ - public static final String HINT_SHIP_STRATEGY_REPARTITION_RANGE = "SHIP_REPARTITION_RANGE"; - - /** - * Value for the shipping strategy compiler hint that enforces a broadcast strategy on - * the input channel. - * - * @see #HINT_SHIP_STRATEGY - * @see #HINT_SHIP_STRATEGY_FIRST_INPUT - * @see #HINT_SHIP_STRATEGY_SECOND_INPUT - */ - public static final String HINT_SHIP_STRATEGY_BROADCAST = "SHIP_BROADCAST"; - - /** - * Compiler hint key for the operator's local strategy. This String is a key to the operator's - * stub parameters. The corresponding value tells the compiler which local strategy to use to - * process the data inside one partition. - * - *

This hint is ignored by operators that do not have a local strategy (such as Map), - * or by operators that have no choice in their local strategy (such as Cross). - */ - public static final String HINT_LOCAL_STRATEGY = "LOCAL_STRATEGY"; - - /** - * Value for the local strategy compiler hint that enforces a sort based local strategy. - * For example, a Reduce operator will sort the data to group it. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_SORT = "LOCAL_STRATEGY_SORT"; - - /** - * Value for the local strategy compiler hint that enforces a sort based local strategy. - * During sorting a combine method is repeatedly applied to reduce the data volume. For example, - * a Reduce operator will sort the data to group it. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_COMBINING_SORT = "LOCAL_STRATEGY_COMBINING_SORT"; - - /** - * Value for the local strategy compiler hint that enforces a sort merge based local - * strategy on both inputs with subsequent merging of inputs. For example, a Match or - * CoGroup operator will use a sort-merge strategy to find pairs of matching keys. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE = - "LOCAL_STRATEGY_SORT_BOTH_MERGE"; - - /** - * Value for the local strategy compiler hint that enforces a sort merge based local - * strategy. The first input is sorted, the second input is assumed to be sorted. After sorting - * both inputs are merged. For example, a Match or CoGroup operator will use a - * sort-merge strategy to find pairs of matching keys. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_SORT_FIRST_MERGE = - "LOCAL_STRATEGY_SORT_FIRST_MERGE"; - - /** - * Value for the local strategy compiler hint that enforces a sort merge based local - * strategy. The second input is sorted, the first input is assumed to be sorted. After sorting - * both inputs are merged. For example, a Match or CoGroup operator will use a - * sort-merge strategy to find pairs of matching keys. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_SORT_SECOND_MERGE = - "LOCAL_STRATEGY_SORT_SECOND_MERGE"; - - /** - * Value for the local strategy compiler hint that enforces a merge based local strategy. - * Both inputs are assumed to be sorted and are merged. For example, a Match or - * CoGroup operator will use a merge strategy to find pairs of matching keys. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_MERGE = "LOCAL_STRATEGY_MERGE"; - - /** - * Value for the local strategy compiler hint that enforces a hash based local strategy. - * For example, a Match operator will use a hybrid-hash-join strategy to find pairs of - * matching keys. The first input will be used to build the hash table, the second input - * will be used to probe the table. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST = - "LOCAL_STRATEGY_HASH_BUILD_FIRST"; - - /** - * Value for the local strategy compiler hint that enforces a hash based local strategy. - * For example, a Match operator will use a hybrid-hash-join strategy to find pairs of - * matching keys. The second input will be used to build the hash table, the first input - * will be used to probe the table. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND = - "LOCAL_STRATEGY_HASH_BUILD_SECOND"; - - /** - * Value for the local strategy compiler hint that chooses the outer side of the - * nested-loop local strategy. A Cross operator will process the data of the - * first input in the outer-loop of the nested loops. Hence, the data of the first input - * will be is streamed though, while the data of the second input is stored on disk and - * repeatedly read. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST = - "LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST"; - - /** - * Value for the local strategy compiler hint that chooses the outer side of the - * nested-loop local strategy. A Cross operator will process the data of the - * second input in the outer-loop of the nested loops. Hence, the data of the second - * input will be is streamed though, while the data of the first input is stored on disk and - * repeatedly read. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND = - "LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND"; - - /** - * Value for the local strategy compiler hint that chooses the outer side of the - * nested-loop local strategy. A Cross operator will process the data of the - * first input in the outer-loop of the nested loops. Further more, the first input, - * being the outer side, will be processed in blocks, and for each block, the second input, - * being the inner side, will read repeatedly from disk. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST = - "LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST"; - - /** - * Value for the local strategy compiler hint that chooses the outer side of the - * nested-loop local strategy. A Cross operator will process the data of the - * second input in the outer-loop of the nested loops. Further more, the second input, - * being the outer side, will be processed in blocks, and for each block, the first input, being - * the inner side, will read repeatedly from disk. - * - * @see #HINT_LOCAL_STRATEGY - */ - public static final String HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND = - "LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND"; - - /** The log handle that is used by the compiler to log messages. */ - public static final Logger LOG = LoggerFactory.getLogger(Optimizer.class); - - // ------------------------------------------------------------------------ - // Members - // ------------------------------------------------------------------------ - - /** - * The statistics object used to obtain statistics, such as input sizes, for the cost estimation - * process. - */ - private final DataStatistics statistics; - - /** The cost estimator used by the compiler. */ - private final CostEstimator costEstimator; - - /** The default parallelism for jobs compiled by this compiler. */ - private int defaultParallelism; - - // ------------------------------------------------------------------------ - // Constructor & Setup - // ------------------------------------------------------------------------ - - /** - * Creates a new optimizer instance. The optimizer has no access to statistics about the inputs - * and can hence not determine any properties. It will perform all optimization with unknown - * sizes and hence use only the heuristic cost functions, which result in the selection of the - * most robust execution strategies. - */ - public Optimizer(Configuration config) { - this(null, new DefaultCostEstimator(), config); - } - - /** - * Creates a new optimizer instance that uses the statistics object to determine properties - * about the input. Given those statistics, the optimizer can make better choices for the - * execution strategies. - * - * @param stats The statistics to be used to determine the input properties. - */ - public Optimizer(DataStatistics stats, Configuration config) { - this(stats, new DefaultCostEstimator(), config); - } - - /** - * Creates a new optimizer instance. The optimizer has no access to statistics about the inputs - * and can hence not determine any properties. It will perform all optimization with unknown - * sizes and hence use only the heuristic cost functions, which result in the selection of the - * most robust execution strategies. - * - *

The optimizer uses the given cost estimator to compute the costs of the individual - * operations. - * - * @param estimator The cost estimator to use to cost the individual operations. - */ - public Optimizer(CostEstimator estimator, Configuration config) { - this(null, estimator, config); - } - - /** - * Creates a new optimizer instance that uses the statistics object to determine properties - * about the input. Given those statistics, the optimizer can make better choices for the - * execution strategies. - * - *

The optimizer uses the given cost estimator to compute the costs of the individual - * operations. - * - * @param stats The statistics to be used to determine the input properties. - * @param estimator The CostEstimator to use to cost the individual operations. - */ - public Optimizer(DataStatistics stats, CostEstimator estimator, Configuration config) { - this.statistics = stats; - this.costEstimator = estimator; - - // determine the default parallelism - this.defaultParallelism = config.get(CoreOptions.DEFAULT_PARALLELISM); - - if (defaultParallelism < 1) { - this.defaultParallelism = CoreOptions.DEFAULT_PARALLELISM.defaultValue(); - LOG.warn( - "Config value {} for option {} is invalid. Ignoring and using a value of {}.", - defaultParallelism, - CoreOptions.DEFAULT_PARALLELISM.key(), - defaultParallelism); - } - } - - // ------------------------------------------------------------------------ - // Getters / Setters - // ------------------------------------------------------------------------ - - public int getDefaultParallelism() { - return defaultParallelism; - } - - public void setDefaultParallelism(int defaultParallelism) { - if (defaultParallelism > 0) { - this.defaultParallelism = defaultParallelism; - } else { - throw new IllegalArgumentException("Default parallelism cannot be zero or negative."); - } - } - - // ------------------------------------------------------------------------ - // Compilation - // ------------------------------------------------------------------------ - - /** - * Translates the given program to an OptimizedPlan, where all nodes have their local strategy - * assigned and all channels have a shipping strategy assigned. - * - *

For more details on the optimization phase, see the comments for {@link - * #compile(org.apache.flink.api.common.Plan, - * org.apache.flink.optimizer.postpass.OptimizerPostPass)}. - * - * @param program The program to be translated. - * @return The optimized plan. - * @throws CompilerException Thrown, if the plan is invalid or the optimizer encountered an - * inconsistent situation during the compilation process. - */ - public OptimizedPlan compile(Plan program) throws CompilerException { - final OptimizerPostPass postPasser = getPostPassFromPlan(program); - return compile(program, postPasser); - } - - /** - * Translates the given program to an OptimizedPlan. The optimized plan describes for each - * operator which strategy to use (such as hash join versus sort-merge join), what data exchange - * method to use (local pipe forward, shuffle, broadcast), what exchange mode to use (pipelined, - * batch), where to cache intermediate results, etc, - * - *

The optimization happens in multiple phases: - * - *

    - *
  1. Create optimizer dag implementation of the program. - *

    OptimizerNode representations of the PACTs, assign parallelism and compute - * size estimates. - *

  2. Compute interesting properties and auxiliary structures. - *
  3. Enumerate plan alternatives. This cannot be done in the same step as the interesting - * property computation (as opposed to the Database approaches), because we support plans - * that are not trees. - *
- * - * @param program The program to be translated. - * @param postPasser The function to be used for post passing the optimizer's plan and setting - * the data type specific serialization routines. - * @return The optimized plan. - * @throws CompilerException Thrown, if the plan is invalid or the optimizer encountered an - * inconsistent situation during the compilation process. - */ - private OptimizedPlan compile(Plan program, OptimizerPostPass postPasser) - throws CompilerException { - if (program == null || postPasser == null) { - throw new NullPointerException(); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Beginning compilation of program '" + program.getJobName() + '\''); - } - - final ExecutionMode defaultDataExchangeMode = - program.getExecutionConfig().getExecutionMode(); - - final int defaultParallelism = - program.getDefaultParallelism() > 0 - ? program.getDefaultParallelism() - : this.defaultParallelism; - - // log the default settings - LOG.debug("Using a default parallelism of {}", defaultParallelism); - LOG.debug("Using default data exchange mode {}", defaultDataExchangeMode); - - // the first step in the compilation is to create the optimizer plan representation - // this step does the following: - // 1) It creates an optimizer plan node for each operator - // 2) It connects them via channels - // 3) It looks for hints about local strategies and channel types and - // sets the types and strategies accordingly - // 4) It makes estimates about the data volume of the data sources and - // propagates those estimates through the plan - - GraphCreatingVisitor graphCreator = - new GraphCreatingVisitor(defaultParallelism, defaultDataExchangeMode); - program.accept(graphCreator); - - // if we have a plan with multiple data sinks, add logical optimizer nodes that have two - // data-sinks as children - // each until we have only a single root node. This allows to transparently deal with the - // nodes with - // multiple outputs - OptimizerNode rootNode; - if (graphCreator.getSinks().size() == 1) { - rootNode = graphCreator.getSinks().get(0); - } else if (graphCreator.getSinks().size() > 1) { - Iterator iter = graphCreator.getSinks().iterator(); - rootNode = iter.next(); - - while (iter.hasNext()) { - rootNode = new SinkJoiner(rootNode, iter.next()); - } - } else { - throw new CompilerException("Bug: The optimizer plan representation has no sinks."); - } - - // now that we have all nodes created and recorded which ones consume memory, tell the nodes - // their minimal - // guaranteed memory, for further cost estimations. We assume an equal distribution of - // memory among consumer tasks - rootNode.accept(new IdAndEstimatesVisitor(this.statistics)); - - // We need to enforce that union nodes always forward their output to their successor. - // Any partitioning must be either pushed before or done after the union, but not on the - // union's output. - UnionParallelismAndForwardEnforcer unionEnforcer = new UnionParallelismAndForwardEnforcer(); - rootNode.accept(unionEnforcer); - - // We are dealing with operator DAGs, rather than operator trees. - // That requires us to deviate at some points from the classical DB optimizer algorithms. - // This step builds auxiliary structures to help track branches and joins in the DAG - BranchesVisitor branchingVisitor = new BranchesVisitor(); - rootNode.accept(branchingVisitor); - - // Propagate the interesting properties top-down through the graph - InterestingPropertyVisitor propsVisitor = - new InterestingPropertyVisitor(this.costEstimator); - rootNode.accept(propsVisitor); - - // perform a sanity check: the root may not have any unclosed branches - if (rootNode.getOpenBranches() != null && rootNode.getOpenBranches().size() > 0) { - throw new CompilerException( - "Bug: Logic for branching plans (non-tree plans) has an error, and does not " - + "track the re-joining of branches correctly."); - } - - // the final step is now to generate the actual plan alternatives - List bestPlan = rootNode.getAlternativePlans(this.costEstimator); - - if (bestPlan.size() != 1) { - throw new CompilerException("Error in compiler: more than one best plan was created!"); - } - - // check if the best plan's root is a data sink (single sink plan) - // if so, directly take it. if it is a sink joiner node, get its contained sinks - PlanNode bestPlanRoot = bestPlan.get(0); - List bestPlanSinks = new ArrayList(4); - - if (bestPlanRoot instanceof SinkPlanNode) { - bestPlanSinks.add((SinkPlanNode) bestPlanRoot); - } else if (bestPlanRoot instanceof SinkJoinerPlanNode) { - ((SinkJoinerPlanNode) bestPlanRoot).getDataSinks(bestPlanSinks); - } - - // finalize the plan - OptimizedPlan plan = - new PlanFinalizer().createFinalPlan(bestPlanSinks, program.getJobName(), program); - - plan.accept(new BinaryUnionReplacer()); - - plan.accept(new RangePartitionRewriter(plan)); - - // post pass the plan. this is the phase where the serialization and comparator code is set - postPasser.postPass(plan); - - return plan; - } - - /** - * This function performs only the first step to the compilation process - the creation of the - * optimizer representation of the plan. No estimations or enumerations of alternatives are done - * here. - * - * @param program The plan to generate the optimizer representation for. - * @return The optimizer representation of the plan, as a collection of all data sinks from the - * plan can be traversed. - */ - public static List createPreOptimizedPlan(Plan program) { - GraphCreatingVisitor graphCreator = new GraphCreatingVisitor(1, null); - program.accept(graphCreator); - return graphCreator.getSinks(); - } - - // ------------------------------------------------------------------------ - // Miscellaneous - // ------------------------------------------------------------------------ - - private OptimizerPostPass getPostPassFromPlan(Plan program) { - final String className = program.getPostPassClassName(); - if (className == null) { - throw new CompilerException("Optimizer Post Pass class description is null"); - } - try { - Class clazz = - Class.forName(className).asSubclass(OptimizerPostPass.class); - try { - return InstantiationUtil.instantiate(clazz, OptimizerPostPass.class); - } catch (RuntimeException rtex) { - // unwrap the source exception - if (rtex.getCause() != null) { - throw new CompilerException( - "Cannot instantiate optimizer post pass: " + rtex.getMessage(), - rtex.getCause()); - } else { - throw rtex; - } - } - } catch (ClassNotFoundException cnfex) { - throw new CompilerException( - "Cannot load Optimizer post-pass class '" + className + "'.", cnfex); - } catch (ClassCastException ccex) { - throw new CompilerException( - "Class '" + className + "' is not an optimizer post-pass.", ccex); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java deleted file mode 100644 index fe46a33f60a67..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/CostEstimator.java +++ /dev/null @@ -1,264 +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.optimizer.costs; - -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.EstimateProvider; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.PlanNode; - -import java.util.Iterator; - -/** - * Abstract base class for a cost estimator. Defines cost estimation methods and implements the - * basic work method that computes the cost of an operator by adding input shipping cost, input - * local cost, and driver cost. - */ -public abstract class CostEstimator { - - public abstract void addRandomPartitioningCost(EstimateProvider estimates, Costs costs); - - public abstract void addHashPartitioningCost(EstimateProvider estimates, Costs costs); - - public abstract void addRangePartitionCost(EstimateProvider estimates, Costs costs); - - public abstract void addBroadcastCost( - EstimateProvider estimates, int replicationFactor, Costs costs); - - // ------------------------------------------------------------------------ - - public abstract void addFileInputCost(long fileSizeInBytes, Costs costs); - - public abstract void addLocalSortCost(EstimateProvider estimates, Costs costs); - - public abstract void addLocalMergeCost( - EstimateProvider estimates1, EstimateProvider estimates2, Costs costs, int costWeight); - - public abstract void addHybridHashCosts( - EstimateProvider buildSide, EstimateProvider probeSide, Costs costs, int costWeight); - - public abstract void addCachedHybridHashCosts( - EstimateProvider buildSide, EstimateProvider probeSide, Costs costs, int costWeight); - - public abstract void addStreamedNestedLoopsCosts( - EstimateProvider outerSide, - EstimateProvider innerSide, - long bufferSize, - Costs costs, - int costWeight); - - public abstract void addBlockNestedLoopsCosts( - EstimateProvider outerSide, - EstimateProvider innerSide, - long blockSize, - Costs costs, - int costWeight); - - // ------------------------------------------------------------------------ - - public abstract void addArtificialDamCost( - EstimateProvider estimates, long bufferSize, Costs costs); - - // ------------------------------------------------------------------------ - - /** - * This method computes the cost of an operator. The cost is composed of cost for input - * shipping, locally processing an input, and running the operator. - * - *

It requires at least that all inputs are set and have a proper ship strategy set, which is - * not equal to NONE. - * - * @param n The node to compute the costs for. - */ - public void costOperator(PlanNode n) { - // initialize costs objects with no costs - final Costs totalCosts = new Costs(); - final long availableMemory = n.getGuaranteedAvailableMemory(); - - // add the shipping strategy costs - for (Channel channel : n.getInputs()) { - final Costs costs = new Costs(); - - // Plans that apply the same strategies, but at different points - // are equally expensive. For example, if a partitioning can be - // pushed below a Map function there is often no difference in plan - // costs between the pushed down version and the version that partitions - // after the Mapper. However, in those cases, we want the expensive - // strategy to appear later in the plan, as data reduction often occurs - // by large factors, while blowup is rare and typically by smaller fractions. - // We achieve this by adding a penalty to small penalty to the FORWARD strategy, - // weighted by the current plan depth (steps to the earliest data source). - // that way, later FORWARDS are more expensive than earlier forwards. - // Note that this only applies to the heuristic costs. - - switch (channel.getShipStrategy()) { - case NONE: - throw new CompilerException( - "Cannot determine costs: Shipping strategy has not been set for an input."); - case FORWARD: - // costs.addHeuristicNetworkCost(channel.getMaxDepth()); - break; - case PARTITION_RANDOM: - addRandomPartitioningCost(channel, costs); - break; - case PARTITION_HASH: - case PARTITION_CUSTOM: - addHashPartitioningCost(channel, costs); - break; - case PARTITION_RANGE: - addRangePartitionCost(channel, costs); - break; - case BROADCAST: - addBroadcastCost(channel, channel.getReplicationFactor(), costs); - break; - case PARTITION_FORCED_REBALANCE: - addRandomPartitioningCost(channel, costs); - break; - default: - throw new CompilerException( - "Unknown shipping strategy for input: " + channel.getShipStrategy()); - } - - switch (channel.getLocalStrategy()) { - case NONE: - break; - case SORT: - case COMBININGSORT: - addLocalSortCost(channel, costs); - break; - default: - throw new CompilerException( - "Unsupported local strategy for input: " + channel.getLocalStrategy()); - } - - if (channel.getTempMode() != null && channel.getTempMode() != TempMode.NONE) { - addArtificialDamCost(channel, 0, costs); - } - - // adjust with the cost weight factor - if (channel.isOnDynamicPath()) { - costs.multiplyWith(channel.getCostWeight()); - } - - totalCosts.addCosts(costs); - } - - Channel firstInput = null; - Channel secondInput = null; - Costs driverCosts = new Costs(); - int costWeight = 1; - - // adjust with the cost weight factor - if (n.isOnDynamicPath()) { - costWeight = n.getCostWeight(); - } - - // get the inputs, if we have some - { - Iterator channels = n.getInputs().iterator(); - if (channels.hasNext()) { - firstInput = channels.next(); - } - if (channels.hasNext()) { - secondInput = channels.next(); - } - } - - // determine the local costs - switch (n.getDriverStrategy()) { - case NONE: - case UNARY_NO_OP: - case BINARY_NO_OP: - case MAP: - case MAP_PARTITION: - case FLAT_MAP: - - case ALL_GROUP_REDUCE: - case ALL_REDUCE: - // this operations does not do any actual grouping, since every element is in the - // same single group - - case CO_GROUP: - case CO_GROUP_RAW: - case SORTED_GROUP_REDUCE: - case SORTED_REDUCE: - // grouping or co-grouping over sorted streams for free - - case SORTED_GROUP_COMBINE: - // partial grouping is always local and main memory resident. we should add a - // relative cpu cost at some point - - // partial grouping is always local and main memory resident. we should add a - // relative cpu cost at some point - case ALL_GROUP_COMBINE: - - case UNION: - // pipelined local union is for free - - break; - case INNER_MERGE: - case FULL_OUTER_MERGE: - case LEFT_OUTER_MERGE: - case RIGHT_OUTER_MERGE: - addLocalMergeCost(firstInput, secondInput, driverCosts, costWeight); - break; - case HYBRIDHASH_BUILD_FIRST: - case RIGHT_HYBRIDHASH_BUILD_FIRST: - case LEFT_HYBRIDHASH_BUILD_FIRST: - case FULL_OUTER_HYBRIDHASH_BUILD_FIRST: - addHybridHashCosts(firstInput, secondInput, driverCosts, costWeight); - break; - case HYBRIDHASH_BUILD_SECOND: - case LEFT_HYBRIDHASH_BUILD_SECOND: - case RIGHT_HYBRIDHASH_BUILD_SECOND: - case FULL_OUTER_HYBRIDHASH_BUILD_SECOND: - addHybridHashCosts(secondInput, firstInput, driverCosts, costWeight); - break; - case HYBRIDHASH_BUILD_FIRST_CACHED: - addCachedHybridHashCosts(firstInput, secondInput, driverCosts, costWeight); - break; - case HYBRIDHASH_BUILD_SECOND_CACHED: - addCachedHybridHashCosts(secondInput, firstInput, driverCosts, costWeight); - break; - case NESTEDLOOP_BLOCKED_OUTER_FIRST: - addBlockNestedLoopsCosts( - firstInput, secondInput, availableMemory, driverCosts, costWeight); - break; - case NESTEDLOOP_BLOCKED_OUTER_SECOND: - addBlockNestedLoopsCosts( - secondInput, firstInput, availableMemory, driverCosts, costWeight); - break; - case NESTEDLOOP_STREAMED_OUTER_FIRST: - addStreamedNestedLoopsCosts( - firstInput, secondInput, availableMemory, driverCosts, costWeight); - break; - case NESTEDLOOP_STREAMED_OUTER_SECOND: - addStreamedNestedLoopsCosts( - secondInput, firstInput, availableMemory, driverCosts, costWeight); - break; - default: - throw new CompilerException( - "Unknown local strategy: " + n.getDriverStrategy().name()); - } - - totalCosts.addCosts(driverCosts); - n.setCosts(totalCosts); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/Costs.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/Costs.java deleted file mode 100644 index 3e3e1f1eaaedc..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/Costs.java +++ /dev/null @@ -1,502 +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.optimizer.costs; - -/** - * Simple class to represent the costs of an operation. The costs are currently tracking, network, - * I/O and CPU costs. - * - *

Costs are composed of two parts of cost contributors: - * - *

    - *
  1. Quantifiable costs. Those costs are used when estimates are available and track a - * quantifiable measure, such as the number of bytes for network or I/O - *
  2. Heuristic costs. Those costs are used when no estimates are available. They can be used to - * track that an operator used a special operation which is heuristically considered more - * expensive than another operation. - *
- * - *

The quantifiable costs may frequently be unknown, which is represented by a {@code -1} as a - * value for the unknown components of the cost. In that case, all operations' costs are unknown and - * hence it is not decidable which operation to favor during pruning. In that case, the heuristic - * costs should contain a value to make sure that operators with different strategies are - * comparable, even in the absence of estimates. The heuristic costs are hence the system's - * mechanism of realizing pruning heuristics that favor some operations over others. - */ -public class Costs implements Comparable, Cloneable { - - public static final double UNKNOWN = -1; - - private double networkCost; // network cost, in transferred bytes - - private double diskCost; // bytes to be written and read, in bytes - - private double cpuCost; // CPU costs - - private double heuristicNetworkCost; - - private double heuristicDiskCost; - - private double heuristicCpuCost; - - // -------------------------------------------------------------------------------------------- - - /** Default constructor. Initializes all costs to 0; */ - public Costs() {} - - /** - * Creates a new costs object using the given values for the network and storage cost. - * - * @param networkCost The network cost, in bytes to be transferred. - * @param diskCost The cost for disk, in bytes to be written and read. - */ - public Costs(double networkCost, double diskCost) { - setNetworkCost(networkCost); - setDiskCost(diskCost); - } - - /** - * Creates a new costs object using the given values for the network and storage cost. - * - * @param networkCost The network cost, in bytes to be transferred. - * @param diskCost The cost for disk, in bytes to be written and read. - * @param cpuCost The cost for CPU operations. - */ - public Costs(double networkCost, double diskCost, double cpuCost) { - setNetworkCost(networkCost); - setDiskCost(diskCost); - setCpuCost(cpuCost); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the network cost. - * - * @return The network cost, in bytes to be transferred. - */ - public double getNetworkCost() { - return networkCost; - } - - /** - * Sets the network cost for this Costs object. - * - * @param bytes The network cost to set, in bytes to be transferred. - */ - public void setNetworkCost(double bytes) { - if (bytes == UNKNOWN || bytes >= 0) { - this.networkCost = bytes; - } else { - throw new IllegalArgumentException(); - } - } - - /** - * Adds the costs for network to the current network costs for this Costs object. - * - * @param bytes The network cost to add, in bytes to be transferred. - */ - public void addNetworkCost(double bytes) { - this.networkCost = (this.networkCost < 0 || bytes < 0) ? UNKNOWN : this.networkCost + bytes; - } - - /** - * Gets the costs for disk. - * - * @return The disk cost, in bytes to be written and read. - */ - public double getDiskCost() { - return diskCost; - } - - /** - * Sets the costs for disk for this Costs object. - * - * @param bytes The disk cost to set, in bytes to be written and read. - */ - public void setDiskCost(double bytes) { - if (bytes == UNKNOWN || bytes >= 0) { - this.diskCost = bytes; - } else { - throw new IllegalArgumentException(); - } - } - - /** - * Adds the costs for disk to the current disk costs for this Costs object. - * - * @param bytes The disk cost to add, in bytes to be written and read. - */ - public void addDiskCost(double bytes) { - this.diskCost = (this.diskCost < 0 || bytes < 0) ? UNKNOWN : this.diskCost + bytes; - } - - /** - * Gets the cost for the CPU. - * - * @return The CPU Cost. - */ - public double getCpuCost() { - return this.cpuCost; - } - - /** - * Sets the cost for the CPU. - * - * @param cost The CPU Cost. - */ - public void setCpuCost(double cost) { - if (cost == UNKNOWN || cost >= 0) { - this.cpuCost = cost; - } else { - throw new IllegalArgumentException(); - } - } - - /** - * Adds the given CPU cost to the current CPU cost for this Costs object. - * - * @param cost The CPU cost to add. - */ - public void addCpuCost(double cost) { - this.cpuCost = (this.cpuCost < 0 || cost < 0) ? UNKNOWN : this.cpuCost + cost; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the heuristic network cost. - * - * @return The heuristic network cost, in bytes to be transferred. - */ - public double getHeuristicNetworkCost() { - return this.heuristicNetworkCost; - } - - /** - * Sets the heuristic network cost for this Costs object. - * - * @param cost The heuristic network cost to set, in bytes to be transferred. - */ - public void setHeuristicNetworkCost(double cost) { - if (cost <= 0) { - throw new IllegalArgumentException("Heuristic costs must be positive."); - } - this.heuristicNetworkCost = cost; - } - - /** - * Adds the heuristic costs for network to the current heuristic network costs for this Costs - * object. - * - * @param cost The heuristic network cost to add. - */ - public void addHeuristicNetworkCost(double cost) { - if (cost <= 0) { - throw new IllegalArgumentException("Heuristic costs must be positive."); - } - this.heuristicNetworkCost += cost; - // check for overflow - if (this.heuristicNetworkCost < 0) { - this.heuristicNetworkCost = Double.MAX_VALUE; - } - } - - /** - * Gets the heuristic costs for disk. - * - * @return The heuristic disk cost. - */ - public double getHeuristicDiskCost() { - return this.heuristicDiskCost; - } - - /** - * Sets the heuristic costs for disk for this Costs object. - * - * @param cost The heuristic disk cost to set. - */ - public void setHeuristicDiskCost(double cost) { - if (cost <= 0) { - throw new IllegalArgumentException("Heuristic costs must be positive."); - } - this.heuristicDiskCost = cost; - } - - /** - * Adds the heuristic costs for disk to the current heuristic disk costs for this Costs object. - * - * @param cost The heuristic disk cost to add. - */ - public void addHeuristicDiskCost(double cost) { - if (cost <= 0) { - throw new IllegalArgumentException("Heuristic costs must be positive."); - } - this.heuristicDiskCost += cost; - // check for overflow - if (this.heuristicDiskCost < 0) { - this.heuristicDiskCost = Double.MAX_VALUE; - } - } - - /** - * Gets the heuristic cost for the CPU. - * - * @return The heuristic CPU Cost. - */ - public double getHeuristicCpuCost() { - return this.heuristicCpuCost; - } - - /** - * Sets the heuristic cost for the CPU. - * - * @param cost The heuristic CPU Cost. - */ - public void setHeuristicCpuCost(double cost) { - if (cost <= 0) { - throw new IllegalArgumentException("Heuristic costs must be positive."); - } - this.heuristicCpuCost = cost; - } - - /** - * Adds the given heuristic CPU cost to the current heuristic CPU cost for this Costs object. - * - * @param cost The heuristic CPU cost to add. - */ - public void addHeuristicCpuCost(double cost) { - if (cost <= 0) { - throw new IllegalArgumentException("Heuristic costs must be positive."); - } - this.heuristicCpuCost += cost; - // check for overflow - if (this.heuristicCpuCost < 0) { - this.heuristicCpuCost = Double.MAX_VALUE; - } - } - - // -------------------------------------------------------------------------------------------- - - /** - * Adds the given costs to these costs. If for one of the different cost components (network, - * disk), the costs are unknown, the resulting costs will be unknown. - * - * @param other The costs to add. - */ - public void addCosts(Costs other) { - // ---------- quantifiable costs ---------- - if (this.networkCost == UNKNOWN || other.networkCost == UNKNOWN) { - this.networkCost = UNKNOWN; - } else { - this.networkCost += other.networkCost; - } - - if (this.diskCost == UNKNOWN || other.diskCost == UNKNOWN) { - this.diskCost = UNKNOWN; - } else { - this.diskCost += other.diskCost; - } - - if (this.cpuCost == UNKNOWN || other.cpuCost == UNKNOWN) { - this.cpuCost = UNKNOWN; - } else { - this.cpuCost += other.cpuCost; - } - - // ---------- heuristic costs ---------- - - this.heuristicNetworkCost += other.heuristicNetworkCost; - this.heuristicDiskCost += other.heuristicDiskCost; - this.heuristicCpuCost += other.heuristicCpuCost; - } - - /** - * Subtracts the given costs from these costs. If the given costs are unknown, then these costs - * are remain unchanged. - * - * @param other The costs to subtract. - */ - public void subtractCosts(Costs other) { - if (this.networkCost != UNKNOWN && other.networkCost != UNKNOWN) { - this.networkCost -= other.networkCost; - if (this.networkCost < 0) { - throw new IllegalArgumentException("Cannot subtract more cost then there is."); - } - } - if (this.diskCost != UNKNOWN && other.diskCost != UNKNOWN) { - this.diskCost -= other.diskCost; - if (this.diskCost < 0) { - throw new IllegalArgumentException("Cannot subtract more cost then there is."); - } - } - if (this.cpuCost != UNKNOWN && other.cpuCost != UNKNOWN) { - this.cpuCost -= other.cpuCost; - if (this.cpuCost < 0) { - throw new IllegalArgumentException("Cannot subtract more cost then there is."); - } - } - - // ---------- relative costs ---------- - - this.heuristicNetworkCost -= other.heuristicNetworkCost; - if (this.heuristicNetworkCost < 0) { - throw new IllegalArgumentException("Cannot subtract more cost then there is."); - } - this.heuristicDiskCost -= other.heuristicDiskCost; - if (this.heuristicDiskCost < 0) { - throw new IllegalArgumentException("Cannot subtract more cost then there is."); - } - this.heuristicCpuCost -= other.heuristicCpuCost; - if (this.heuristicCpuCost < 0) { - throw new IllegalArgumentException("Cannot subtract more cost then there is."); - } - } - - public void multiplyWith(int factor) { - this.networkCost = this.networkCost < 0 ? -1 : this.networkCost * factor; - this.diskCost = this.diskCost < 0 ? -1 : this.diskCost * factor; - this.cpuCost = this.cpuCost < 0 ? -1 : this.cpuCost * factor; - this.heuristicNetworkCost = - this.heuristicNetworkCost < 0 ? -1 : this.heuristicNetworkCost * factor; - this.heuristicDiskCost = this.heuristicDiskCost < 0 ? -1 : this.heuristicDiskCost * factor; - this.heuristicCpuCost = this.heuristicCpuCost < 0 ? -1 : this.heuristicCpuCost * factor; - } - - public void divideBy(int factor) { - this.networkCost = this.networkCost < 0 ? -1 : this.networkCost / factor; - this.diskCost = this.diskCost < 0 ? -1 : this.diskCost / factor; - this.cpuCost = this.cpuCost < 0 ? -1 : this.cpuCost / factor; - this.heuristicNetworkCost = - this.heuristicNetworkCost < 0 ? -1 : this.heuristicNetworkCost / factor; - this.heuristicDiskCost = this.heuristicDiskCost < 0 ? -1 : this.heuristicDiskCost / factor; - this.heuristicCpuCost = this.heuristicCpuCost < 0 ? -1 : this.heuristicCpuCost / factor; - } - - // -------------------------------------------------------------------------------------------- - - /** - * The order of comparison is: network first, then disk, then CPU. The comparison here happens - * each time primarily after the heuristic costs, then after the quantifiable costs. - * - * @see java.lang.Comparable#compareTo(java.lang.Object) - */ - @Override - public int compareTo(Costs o) { - // check the network cost. if we have actual costs on both, use them, otherwise use the - // heuristic costs. - if (this.networkCost != UNKNOWN && o.networkCost != UNKNOWN) { - if (this.networkCost != o.networkCost) { - return this.networkCost < o.networkCost ? -1 : 1; - } - } else if (this.heuristicNetworkCost < o.heuristicNetworkCost) { - return -1; - } else if (this.heuristicNetworkCost > o.heuristicNetworkCost) { - return 1; - } - - // next, check the disk cost. again, if we have actual costs on both, use them, otherwise - // use the heuristic costs. - if (this.diskCost != UNKNOWN && o.diskCost != UNKNOWN) { - if (this.diskCost != o.diskCost) { - return this.diskCost < o.diskCost ? -1 : 1; - } - } else if (this.heuristicDiskCost < o.heuristicDiskCost) { - return -1; - } else if (this.heuristicDiskCost > o.heuristicDiskCost) { - return 1; - } - - // next, check the CPU cost. again, if we have actual costs on both, use them, otherwise use - // the heuristic costs. - if (this.cpuCost != UNKNOWN && o.cpuCost != UNKNOWN) { - return this.cpuCost < o.cpuCost ? -1 : this.cpuCost > o.cpuCost ? 1 : 0; - } else if (this.heuristicCpuCost < o.heuristicCpuCost) { - return -1; - } else if (this.heuristicCpuCost > o.heuristicCpuCost) { - return 1; - } else { - return 0; - } - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - long cpuCostBits = Double.doubleToLongBits(cpuCost); - long heuristicCpuCostBits = Double.doubleToLongBits(heuristicCpuCost); - long heuristicNetworkCostBits = Double.doubleToLongBits(heuristicNetworkCost); - long heuristicDiskCostBits = Double.doubleToLongBits(heuristicDiskCost); - long networkCostBits = Double.doubleToLongBits(networkCost); - long diskCostBits = Double.doubleToLongBits(diskCost); - - result = prime * result + (int) (cpuCostBits ^ (cpuCostBits >>> 32)); - result = prime * result + (int) (heuristicCpuCostBits ^ (heuristicCpuCostBits >>> 32)); - result = - prime * result - + (int) (heuristicNetworkCostBits ^ (heuristicNetworkCostBits >>> 32)); - result = prime * result + (int) (heuristicDiskCostBits ^ (heuristicDiskCostBits >>> 32)); - result = prime * result + (int) (networkCostBits ^ (networkCostBits >>> 32)); - result = prime * result + (int) (diskCostBits ^ (diskCostBits >>> 32)); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj.getClass() == getClass()) { - final Costs other = (Costs) obj; - return this.networkCost == other.networkCost - & this.diskCost == other.diskCost - & this.cpuCost == other.cpuCost - & this.heuristicNetworkCost == other.heuristicNetworkCost - & this.heuristicDiskCost == other.heuristicDiskCost - & this.heuristicCpuCost == other.heuristicCpuCost; - } else { - return false; - } - } - - @Override - public String toString() { - return "Costs [networkCost=" - + networkCost - + ", diskCost=" - + diskCost - + ", cpuCost=" - + cpuCost - + ", heuristicNetworkCost=" - + heuristicNetworkCost - + ", heuristicDiskCost=" - + heuristicDiskCost - + ", heuristicCpuCost=" - + heuristicCpuCost - + "]"; - } - - @Override - public Costs clone() { - try { - return (Costs) super.clone(); - } catch (CloneNotSupportedException e) { - throw new RuntimeException(e); // should never happen - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/DefaultCostEstimator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/DefaultCostEstimator.java deleted file mode 100644 index eb741dea88ac5..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/costs/DefaultCostEstimator.java +++ /dev/null @@ -1,290 +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.optimizer.costs; - -import org.apache.flink.optimizer.dag.EstimateProvider; - -/** - * A default cost estimator that has access to basic size and cardinality estimates. - * - *

This estimator works with actual estimates (as far as they are available) and falls back to - * setting relative costs, if no estimates are available. That way, the estimator makes sure that - * plans with different strategies are costed differently, also in the absence of estimates. The - * different relative costs in the absence of estimates represent this estimator's heuristic - * guidance towards certain strategies. - * - *

For robustness reasons, we always assume that the whole data is shipped during a repartition - * step. We deviate from the typical estimate of (n - 1) / n (with n being the - * number of nodes), because for a parallelism of 1, that would yield a shipping of zero bytes. - * While this is usually correct, the runtime scheduling may still choose to move tasks to different - * nodes, so that we do not know that no data is shipped. - */ -public class DefaultCostEstimator extends CostEstimator { - - /** - * The case of the estimation for all relative costs. We heuristically pick a very large data - * volume, which will favor strategies that are less expensive on large data volumes. This is - * robust and - */ - private static final long HEURISTIC_COST_BASE = 1000000000L; - - // The numbers for the CPU effort are rather magic at the moment and should be seen rather - // ordinal - - private static final float MATERIALIZATION_CPU_FACTOR = 1; - - private static final float HASHING_CPU_FACTOR = 4; - - private static final float SORTING_CPU_FACTOR = 9; - - // -------------------------------------------------------------------------------------------- - // Shipping Strategy Cost - // -------------------------------------------------------------------------------------------- - - @Override - public void addRandomPartitioningCost(EstimateProvider estimates, Costs costs) { - // conservative estimate: we need ship the whole data over the network to establish the - // partitioning. no disk costs. - final long estOutShipSize = estimates.getEstimatedOutputSize(); - if (estOutShipSize <= 0) { - costs.setNetworkCost(Costs.UNKNOWN); - } else { - costs.addNetworkCost(estOutShipSize); - } - costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE); - } - - @Override - public void addHashPartitioningCost(EstimateProvider estimates, Costs costs) { - // conservative estimate: we need ship the whole data over the network to establish the - // partitioning. no disk costs. - final long estOutShipSize = estimates.getEstimatedOutputSize(); - if (estOutShipSize <= 0) { - costs.setNetworkCost(Costs.UNKNOWN); - } else { - costs.addNetworkCost(estOutShipSize); - } - costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE); - } - - @Override - public void addRangePartitionCost(EstimateProvider estimates, Costs costs) { - final long dataSize = estimates.getEstimatedOutputSize(); - if (dataSize > 0) { - // Assume sampling of 10% of the data and spilling it to disk - final long sampled = (long) (dataSize * 0.1f); - // set shipping costs - costs.addNetworkCost(dataSize + sampled); - } else { - costs.setNetworkCost(Costs.UNKNOWN); - } - - // no costs known. use the same assumption as above on the heuristic costs - final long sampled = (long) (HEURISTIC_COST_BASE * 0.1f); - costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE + sampled); - costs.addHeuristicDiskCost(2 * sampled); - } - - @Override - public void addBroadcastCost(EstimateProvider estimates, int replicationFactor, Costs costs) { - // if our replication factor is negative, we cannot calculate broadcast costs - if (replicationFactor <= 0) { - throw new IllegalArgumentException( - "The replication factor of must be larger than zero."); - } - - if (replicationFactor > 0) { - // assumption: we need ship the whole data over the network to each node. - final long estOutShipSize = estimates.getEstimatedOutputSize(); - if (estOutShipSize <= 0) { - costs.setNetworkCost(Costs.UNKNOWN); - } else { - costs.addNetworkCost(replicationFactor * estOutShipSize); - } - costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * 10 * replicationFactor); - } else { - costs.addHeuristicNetworkCost(HEURISTIC_COST_BASE * 1000); - } - } - - // -------------------------------------------------------------------------------------------- - // Local Strategy Cost - // -------------------------------------------------------------------------------------------- - - @Override - public void addFileInputCost(long fileSizeInBytes, Costs costs) { - if (fileSizeInBytes >= 0) { - costs.addDiskCost(fileSizeInBytes); - } else { - costs.setDiskCost(Costs.UNKNOWN); - } - costs.addHeuristicDiskCost(HEURISTIC_COST_BASE); - } - - @Override - public void addLocalSortCost(EstimateProvider estimates, Costs costs) { - final long s = estimates.getEstimatedOutputSize(); - // we assume a two phase merge sort, so all in all 2 I/O operations per block - if (s <= 0) { - costs.setDiskCost(Costs.UNKNOWN); - costs.setCpuCost(Costs.UNKNOWN); - } else { - costs.addDiskCost(2 * s); - costs.addCpuCost((long) (s * SORTING_CPU_FACTOR)); - } - costs.addHeuristicDiskCost(2 * HEURISTIC_COST_BASE); - costs.addHeuristicCpuCost((long) (HEURISTIC_COST_BASE * SORTING_CPU_FACTOR)); - } - - @Override - public void addLocalMergeCost( - EstimateProvider input1, EstimateProvider input2, Costs costs, int costWeight) { - // costs nothing. the very rarely incurred cost for a spilling block nested loops join in - // the - // presence of massively re-occurring duplicate keys is ignored, because cannot be assessed - } - - @Override - public void addHybridHashCosts( - EstimateProvider buildSideInput, - EstimateProvider probeSideInput, - Costs costs, - int costWeight) { - long bs = buildSideInput.getEstimatedOutputSize(); - long ps = probeSideInput.getEstimatedOutputSize(); - - if (bs > 0 && ps > 0) { - long overall = 2 * bs + ps; - costs.addDiskCost(overall); - costs.addCpuCost((long) (overall * HASHING_CPU_FACTOR)); - } else { - costs.setDiskCost(Costs.UNKNOWN); - costs.setCpuCost(Costs.UNKNOWN); - } - costs.addHeuristicDiskCost(2 * HEURISTIC_COST_BASE); - costs.addHeuristicCpuCost((long) (2 * HEURISTIC_COST_BASE * HASHING_CPU_FACTOR)); - - // cost weight applies to everything - costs.multiplyWith(costWeight); - } - - /** - * Calculates the costs for the cached variant of the hybrid hash join. We are assuming by - * default that half of the cached hash table fit into memory. - */ - @Override - public void addCachedHybridHashCosts( - EstimateProvider buildSideInput, - EstimateProvider probeSideInput, - Costs costs, - int costWeight) { - if (costWeight < 1) { - throw new IllegalArgumentException("The cost weight must be at least one."); - } - - long bs = buildSideInput.getEstimatedOutputSize(); - long ps = probeSideInput.getEstimatedOutputSize(); - - if (bs > 0 && ps > 0) { - long overall = 2 * bs + costWeight * ps; - costs.addDiskCost(overall); - costs.addCpuCost((long) (overall * HASHING_CPU_FACTOR)); - } else { - costs.setDiskCost(Costs.UNKNOWN); - costs.setCpuCost(Costs.UNKNOWN); - } - - // one time the build side plus cost-weight time the probe side - costs.addHeuristicDiskCost((1 + costWeight) * HEURISTIC_COST_BASE); - costs.addHeuristicCpuCost( - (long) ((1 + costWeight) * HEURISTIC_COST_BASE * HASHING_CPU_FACTOR)); - } - - @Override - public void addStreamedNestedLoopsCosts( - EstimateProvider outerSide, - EstimateProvider innerSide, - long bufferSize, - Costs costs, - int costWeight) { - long is = innerSide.getEstimatedOutputSize(); - long oc = outerSide.getEstimatedNumRecords(); - - if (is > 0 && oc >= 0) { - // costs, if the inner side cannot be cached - if (is > bufferSize) { - costs.addDiskCost(oc * is); - } - costs.addCpuCost((long) (oc * is * MATERIALIZATION_CPU_FACTOR)); - } else { - costs.setDiskCost(Costs.UNKNOWN); - costs.setCpuCost(Costs.UNKNOWN); - } - - // hack: assume 100k loops (should be expensive enough) - costs.addHeuristicDiskCost(HEURISTIC_COST_BASE * 100000); - costs.addHeuristicCpuCost( - (long) (HEURISTIC_COST_BASE * 100000 * MATERIALIZATION_CPU_FACTOR)); - costs.multiplyWith(costWeight); - } - - @Override - public void addBlockNestedLoopsCosts( - EstimateProvider outerSide, - EstimateProvider innerSide, - long blockSize, - Costs costs, - int costWeight) { - long is = innerSide.getEstimatedOutputSize(); - long os = outerSide.getEstimatedOutputSize(); - - if (is > 0 && os > 0) { - long loops = Math.max(os / blockSize, 1); - costs.addDiskCost(loops * is); - costs.addCpuCost((long) (loops * is * MATERIALIZATION_CPU_FACTOR)); - } else { - costs.setDiskCost(Costs.UNKNOWN); - costs.setCpuCost(Costs.UNKNOWN); - } - - // hack: assume 1k loops (much cheaper than the streamed variant!) - costs.addHeuristicDiskCost(HEURISTIC_COST_BASE * 1000); - costs.addHeuristicCpuCost((long) (HEURISTIC_COST_BASE * 1000 * MATERIALIZATION_CPU_FACTOR)); - costs.multiplyWith(costWeight); - } - - // -------------------------------------------------------------------------------------------- - // Damming Cost - // -------------------------------------------------------------------------------------------- - - @Override - public void addArtificialDamCost(EstimateProvider estimates, long bufferSize, Costs costs) { - final long s = estimates.getEstimatedOutputSize(); - // we assume spilling and re-reading - if (s <= 0) { - costs.setDiskCost(Costs.UNKNOWN); - costs.setCpuCost(Costs.UNKNOWN); - } else { - costs.addDiskCost(2 * s); - costs.setCpuCost((long) (s * MATERIALIZATION_CPU_FACTOR)); - } - costs.addHeuristicDiskCost(2 * HEURISTIC_COST_BASE); - costs.addHeuristicCpuCost((long) (HEURISTIC_COST_BASE * MATERIALIZATION_CPU_FACTOR)); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/AbstractPartialSolutionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/AbstractPartialSolutionNode.java deleted file mode 100644 index 39d35a7e1d67f..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/AbstractPartialSolutionNode.java +++ /dev/null @@ -1,106 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.util.Visitor; - -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** - * The optimizer's internal representation of the partial solution that is input to a bulk - * iteration. - */ -public abstract class AbstractPartialSolutionNode extends OptimizerNode { - - protected AbstractPartialSolutionNode(Operator contract) { - super(contract); - } - - // -------------------------------------------------------------------------------------------- - - protected void copyEstimates(OptimizerNode node) { - this.estimatedNumRecords = node.estimatedNumRecords; - this.estimatedOutputSize = node.estimatedOutputSize; - } - - public abstract IterationNode getIterationNode(); - - // -------------------------------------------------------------------------------------------- - - public boolean isOnDynamicPath() { - return true; - } - - public void identifyDynamicPath(int costWeight) { - this.onDynamicPath = true; - this.costWeight = costWeight; - } - - @Override - public List getIncomingConnections() { - return Collections.emptyList(); - } - - @Override - public void setInput( - Map, OptimizerNode> contractToNode, ExecutionMode dataExchangeMode) {} - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // we do nothing here, because the estimates can only be copied from the iteration input - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - // no children, so nothing to compute - } - - @Override - public List getAlternativePlans(CostEstimator estimator) { - if (this.cachedPlans != null) { - return this.cachedPlans; - } else { - throw new IllegalStateException(); - } - } - - @Override - public SemanticProperties getSemanticProperties() { - return new EmptySemanticProperties(); - } - - @Override - protected void readStubAnnotations() {} - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - visitor.postVisit(this); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java deleted file mode 100644 index 949d97ecb7f0a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BinaryUnionNode.java +++ /dev/null @@ -1,400 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.Union; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.BinaryUnionOpDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Set; - -/** The Optimizer representation of a binary Union. */ -public class BinaryUnionNode extends TwoInputNode { - - private Set channelProps; - - public BinaryUnionNode(Union union) { - super(union); - } - - @Override - public void addOutgoingConnection(DagConnection connection) { - // ensure that union nodes have not more than one outgoing connection. - if (this.getOutgoingConnections() != null && this.getOutgoingConnections().size() > 0) { - throw new CompilerException( - "BinaryUnionNode may only have a single outgoing connection."); - } - super.addOutgoingConnection(connection); - } - - @Override - public String getOperatorName() { - return "Union"; - } - - @Override - protected List getPossibleProperties() { - return Collections.emptyList(); - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - long card1 = getFirstPredecessorNode().getEstimatedNumRecords(); - long card2 = getSecondPredecessorNode().getEstimatedNumRecords(); - this.estimatedNumRecords = (card1 < 0 || card2 < 0) ? -1 : card1 + card2; - - long size1 = getFirstPredecessorNode().getEstimatedOutputSize(); - long size2 = getSecondPredecessorNode().getEstimatedOutputSize(); - this.estimatedOutputSize = (size1 < 0 || size2 < 0) ? -1 : size1 + size2; - } - - @Override - public void computeUnionOfInterestingPropertiesFromSuccessors() { - super.computeUnionOfInterestingPropertiesFromSuccessors(); - // clear all local properties, as they are destroyed anyways - getInterestingProperties().getLocalProperties().clear(); - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - final InterestingProperties props = getInterestingProperties(); - - // if no other properties exist, add the pruned trivials back - if (props.getGlobalProperties().isEmpty()) { - props.addGlobalProperties(new RequestedGlobalProperties()); - } - props.addLocalProperties(new RequestedLocalProperties()); - this.input1.setInterestingProperties(props.clone()); - this.input2.setInterestingProperties(props.clone()); - - this.channelProps = props.getGlobalProperties(); - } - - @Override - public List getAlternativePlans(CostEstimator estimator) { - - // check that union has only a single successor - if (this.getOutgoingConnections().size() > 1) { - throw new CompilerException("BinaryUnionNode has more than one successor."); - } - - boolean childrenSkippedDueToReplicatedInput = false; - - // check if we have a cached version - if (this.cachedPlans != null) { - return this.cachedPlans; - } - - // step down to all producer nodes and calculate alternative plans - final List subPlans1 = - getFirstPredecessorNode().getAlternativePlans(estimator); - final List subPlans2 = - getSecondPredecessorNode().getAlternativePlans(estimator); - - List broadcastConnections = getBroadcastConnections(); - if (broadcastConnections != null && broadcastConnections.size() > 0) { - throw new CompilerException("Found BroadcastVariables on a Union operation"); - } - - final ArrayList outputPlans = new ArrayList(); - - final List> broadcastPlanChannels = Collections.emptyList(); - - final BinaryUnionOpDescriptor operator = new BinaryUnionOpDescriptor(); - final RequestedLocalProperties noLocalProps = new RequestedLocalProperties(); - - final ExecutionMode input1Mode = this.input1.getDataExchangeMode(); - final ExecutionMode input2Mode = this.input2.getDataExchangeMode(); - - final int parallelism = getParallelism(); - final int inParallelism1 = getFirstPredecessorNode().getParallelism(); - final int inParallelism2 = getSecondPredecessorNode().getParallelism(); - - final boolean dopChange1 = parallelism != inParallelism1; - final boolean dopChange2 = parallelism != inParallelism2; - - final boolean input1breakPipeline = this.input1.isBreakingPipeline(); - final boolean input2breakPipeline = this.input2.isBreakingPipeline(); - - // enumerate all pairwise combination of the children's plans together with - // all possible operator strategy combination - - // create all candidates - for (PlanNode child1 : subPlans1) { - - if (child1.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if parallelism is not changed - if (dopChange1) { - // can not continue with this child - childrenSkippedDueToReplicatedInput = true; - continue; - } else { - this.input1.setShipStrategy(ShipStrategyType.FORWARD); - } - } - - for (PlanNode child2 : subPlans2) { - - if (child2.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if parallelism is not - // changed - if (dopChange2) { - // can not continue with this child - childrenSkippedDueToReplicatedInput = true; - continue; - } else { - this.input2.setShipStrategy(ShipStrategyType.FORWARD); - } - } - - // check that the children go together. that is the case if they build upon the same - // candidate at the joined branch plan. - if (!areBranchCompatible(child1, child2)) { - continue; - } - - for (RequestedGlobalProperties igps : this.channelProps) { - // create a candidate channel for the first input. mark it cached, if the - // connection says so - Channel c1 = new Channel(child1, this.input1.getMaterializationMode()); - if (this.input1.getShipStrategy() == null) { - // free to choose the ship strategy - igps.parameterizeChannel(c1, dopChange1, input1Mode, input1breakPipeline); - - // if the parallelism changed, make sure that we cancel out properties, - // unless the - // ship strategy preserves/establishes them even under changing parallelisms - if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) { - c1.getGlobalProperties().reset(); - } - } else { - // ship strategy fixed by compiler hint - ShipStrategyType shipStrategy = this.input1.getShipStrategy(); - DataExchangeMode exMode = - DataExchangeMode.select( - input1Mode, shipStrategy, input1breakPipeline); - if (this.keys1 != null) { - c1.setShipStrategy( - this.input1.getShipStrategy(), - this.keys1.toFieldList(), - exMode); - } else { - c1.setShipStrategy(this.input1.getShipStrategy(), exMode); - } - - if (dopChange1) { - c1.adjustGlobalPropertiesForFullParallelismChange(); - } - } - - // create a candidate channel for the second input. mark it cached, if the - // connection says so - Channel c2 = new Channel(child2, this.input2.getMaterializationMode()); - if (this.input2.getShipStrategy() == null) { - // free to choose the ship strategy - igps.parameterizeChannel(c2, dopChange2, input2Mode, input2breakPipeline); - - // if the parallelism changed, make sure that we cancel out properties, - // unless the - // ship strategy preserves/establishes them even under changing parallelisms - if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) { - c2.getGlobalProperties().reset(); - } - } else { - // ship strategy fixed by compiler hint - ShipStrategyType shipStrategy = this.input2.getShipStrategy(); - DataExchangeMode exMode = - DataExchangeMode.select( - input2Mode, shipStrategy, input2breakPipeline); - if (this.keys2 != null) { - c2.setShipStrategy( - this.input2.getShipStrategy(), - this.keys2.toFieldList(), - exMode); - } else { - c2.setShipStrategy(this.input2.getShipStrategy(), exMode); - } - - if (dopChange2) { - c2.adjustGlobalPropertiesForFullParallelismChange(); - } - } - - // get the global properties and clear unique fields (not preserved anyways - // during the union) - GlobalProperties p1 = c1.getGlobalProperties(); - GlobalProperties p2 = c2.getGlobalProperties(); - p1.clearUniqueFieldCombinations(); - p2.clearUniqueFieldCombinations(); - - // adjust the partitioning, if they exist but are not equal. this may happen - // when both channels have a - // partitioning that fulfills the requirements, but both are incompatible. For - // example may a property requirement - // be ANY_PARTITIONING on fields (0) and one channel is range partitioned on - // that field, the other is hash - // partitioned on that field. - if (!igps.isTrivial() && !(p1.equals(p2))) { - if (c1.getShipStrategy() == ShipStrategyType.FORWARD - && c2.getShipStrategy() != ShipStrategyType.FORWARD) { - // adjust c2 to c1 - c2 = c2.clone(); - p1.parameterizeChannel(c2, dopChange2, input2Mode, input2breakPipeline); - } else if (c2.getShipStrategy() == ShipStrategyType.FORWARD - && c1.getShipStrategy() != ShipStrategyType.FORWARD) { - // adjust c1 to c2 - c1 = c1.clone(); - p2.parameterizeChannel(c1, dopChange1, input1Mode, input1breakPipeline); - } else if (c1.getShipStrategy() == ShipStrategyType.FORWARD - && c2.getShipStrategy() == ShipStrategyType.FORWARD) { - boolean adjustC1 = - c1.getEstimatedOutputSize() <= 0 - || c2.getEstimatedOutputSize() <= 0 - || c1.getEstimatedOutputSize() - <= c2.getEstimatedOutputSize(); - if (adjustC1) { - c2 = c2.clone(); - p1.parameterizeChannel( - c2, dopChange2, input2Mode, input2breakPipeline); - } else { - c1 = c1.clone(); - p2.parameterizeChannel( - c1, dopChange1, input1Mode, input1breakPipeline); - } - } else { - // this should never happen, as it implies both realize a different - // strategy, which is - // excluded by the check that the required strategies must match - throw new CompilerException("Bug in Plan Enumeration for Union Node."); - } - } - - instantiate( - operator, - c1, - c2, - broadcastPlanChannels, - outputPlans, - estimator, - igps, - igps, - noLocalProps, - noLocalProps); - } - } - } - - if (outputPlans.isEmpty()) { - if (childrenSkippedDueToReplicatedInput) { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Invalid use of replicated input."); - } else { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Too restrictive plan hints."); - } - } - - // cost and prune the plans - for (PlanNode node : outputPlans) { - estimator.costOperator(node); - } - prunePlanAlternatives(outputPlans); - outputPlans.trimToSize(); - - this.cachedPlans = outputPlans; - return outputPlans; - } - - @Override - protected void readStubAnnotations() {} - - @Override - public SemanticProperties getSemanticProperties() { - return new UnionSemanticProperties(); - } - - @Override - public void computeOutputEstimates(DataStatistics statistics) { - OptimizerNode in1 = getFirstPredecessorNode(); - OptimizerNode in2 = getSecondPredecessorNode(); - - this.estimatedNumRecords = - in1.estimatedNumRecords > 0 && in2.estimatedNumRecords > 0 - ? in1.estimatedNumRecords + in2.estimatedNumRecords - : -1; - this.estimatedOutputSize = - in1.estimatedOutputSize > 0 && in2.estimatedOutputSize > 0 - ? in1.estimatedOutputSize + in2.estimatedOutputSize - : -1; - } - - public static class UnionSemanticProperties implements SemanticProperties { - - private static final long serialVersionUID = 1L; - - @Override - public FieldSet getForwardingTargetFields(int input, int sourceField) { - if (input != 0 && input != 1) { - throw new IndexOutOfBoundsException("Invalid input index for binary union node."); - } - - return new FieldSet(sourceField); - } - - @Override - public int getForwardingSourceField(int input, int targetField) { - if (input != 0 && input != 1) { - throw new IndexOutOfBoundsException(); - } - - return targetField; - } - - @Override - public FieldSet getReadFields(int input) { - if (input != 0 && input != 1) { - throw new IndexOutOfBoundsException(); - } - - return FieldSet.EMPTY_SET; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java deleted file mode 100644 index c585d7e9188ae..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java +++ /dev/null @@ -1,444 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties; -import org.apache.flink.api.common.operators.base.BulkIterationBase; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dag.WorksetIterationNode.SingleRootJoiner; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.NoOpDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.traversals.InterestingPropertyVisitor; -import org.apache.flink.optimizer.util.NoOpUnaryUdfOp; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -/** A node in the optimizer's program representation for a bulk iteration. */ -public class BulkIterationNode extends SingleInputNode implements IterationNode { - - private BulkPartialSolutionNode partialSolution; - - private OptimizerNode terminationCriterion; - - private OptimizerNode nextPartialSolution; - - private DagConnection rootConnection; // connection out of the next partial solution - - private DagConnection - terminationCriterionRootConnection; // connection out of the term. criterion - - private OptimizerNode singleRoot; - - private final int costWeight; - - // -------------------------------------------------------------------------------------------- - - /** - * Creates a new node for the bulk iteration. - * - * @param iteration The bulk iteration the node represents. - */ - public BulkIterationNode(BulkIterationBase iteration) { - super(iteration); - - if (iteration.getMaximumNumberOfIterations() <= 0) { - throw new CompilerException( - "BulkIteration must have a maximum number of iterations specified."); - } - - int numIters = iteration.getMaximumNumberOfIterations(); - - this.costWeight = - (numIters > 0 && numIters < OptimizerNode.MAX_DYNAMIC_PATH_COST_WEIGHT) - ? numIters - : OptimizerNode.MAX_DYNAMIC_PATH_COST_WEIGHT; - } - - // -------------------------------------------------------------------------------------------- - - public BulkIterationBase getIterationContract() { - return (BulkIterationBase) getOperator(); - } - - /** - * Gets the partialSolution from this BulkIterationNode. - * - * @return The partialSolution. - */ - public BulkPartialSolutionNode getPartialSolution() { - return partialSolution; - } - - /** - * Sets the partialSolution for this BulkIterationNode. - * - * @param partialSolution The partialSolution to set. - */ - public void setPartialSolution(BulkPartialSolutionNode partialSolution) { - this.partialSolution = partialSolution; - } - - /** - * Gets the nextPartialSolution from this BulkIterationNode. - * - * @return The nextPartialSolution. - */ - public OptimizerNode getNextPartialSolution() { - return nextPartialSolution; - } - - /** - * Sets the nextPartialSolution for this BulkIterationNode. - * - * @param nextPartialSolution The nextPartialSolution to set. - */ - public void setNextPartialSolution( - OptimizerNode nextPartialSolution, OptimizerNode terminationCriterion) { - - // check if the root of the step function has the same parallelism as the iteration - // or if the step function has any operator at all - if (nextPartialSolution.getParallelism() != getParallelism() - || nextPartialSolution == partialSolution - || nextPartialSolution instanceof BinaryUnionNode) { - // add a no-op to the root to express the re-partitioning - NoOpNode noop = new NoOpNode(); - noop.setParallelism(getParallelism()); - - DagConnection noOpConn = - new DagConnection(nextPartialSolution, noop, ExecutionMode.PIPELINED); - noop.setIncomingConnection(noOpConn); - nextPartialSolution.addOutgoingConnection(noOpConn); - - nextPartialSolution = noop; - } - - this.nextPartialSolution = nextPartialSolution; - this.terminationCriterion = terminationCriterion; - - if (terminationCriterion == null) { - this.singleRoot = nextPartialSolution; - this.rootConnection = new DagConnection(nextPartialSolution, ExecutionMode.PIPELINED); - } else { - // we have a termination criterion - SingleRootJoiner singleRootJoiner = new SingleRootJoiner(); - this.rootConnection = - new DagConnection( - nextPartialSolution, singleRootJoiner, ExecutionMode.PIPELINED); - this.terminationCriterionRootConnection = - new DagConnection( - terminationCriterion, singleRootJoiner, ExecutionMode.PIPELINED); - - singleRootJoiner.setInputs( - this.rootConnection, this.terminationCriterionRootConnection); - - this.singleRoot = singleRootJoiner; - - // add connection to terminationCriterion for interesting properties visitor - terminationCriterion.addOutgoingConnection(terminationCriterionRootConnection); - } - - nextPartialSolution.addOutgoingConnection(rootConnection); - } - - public int getCostWeight() { - return this.costWeight; - } - - public OptimizerNode getSingleRootOfStepFunction() { - return this.singleRoot; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public String getOperatorName() { - return "Bulk Iteration"; - } - - @Override - public SemanticProperties getSemanticProperties() { - return new EmptySemanticProperties(); - } - - protected void readStubAnnotations() {} - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedOutputSize = getPredecessorNode().getEstimatedOutputSize(); - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords(); - } - - // -------------------------------------------------------------------------------------------- - // Properties and Optimization - // -------------------------------------------------------------------------------------------- - - protected List getPossibleProperties() { - return Collections.singletonList(new NoOpDescriptor()); - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - final InterestingProperties intProps = getInterestingProperties().clone(); - - if (this.terminationCriterion != null) { - // first propagate through termination Criterion. since it has no successors, it has no - // interesting properties - this.terminationCriterionRootConnection.setInterestingProperties( - new InterestingProperties()); - this.terminationCriterion.accept(new InterestingPropertyVisitor(estimator)); - } - - // we need to make 2 interesting property passes, because the root of the step function - // needs also - // the interesting properties as generated by the partial solution - - // give our own interesting properties (as generated by the iterations successors) to the - // step function and - // make the first pass - this.rootConnection.setInterestingProperties(intProps); - this.nextPartialSolution.accept(new InterestingPropertyVisitor(estimator)); - - // take the interesting properties of the partial solution and add them to the root - // interesting properties - InterestingProperties partialSolutionIntProps = - this.partialSolution.getInterestingProperties(); - intProps.getGlobalProperties().addAll(partialSolutionIntProps.getGlobalProperties()); - intProps.getLocalProperties().addAll(partialSolutionIntProps.getLocalProperties()); - - // clear all interesting properties to prepare the second traversal - // this clears only the path down from the next partial solution. The paths down - // from the termination criterion (before they meet the paths down from the next partial - // solution) - // remain unaffected by this step - this.rootConnection.clearInterestingProperties(); - this.nextPartialSolution.accept(InterestingPropertiesClearer.INSTANCE); - - // 2nd pass - this.rootConnection.setInterestingProperties(intProps); - this.nextPartialSolution.accept(new InterestingPropertyVisitor(estimator)); - - // now add the interesting properties of the partial solution to the input - final InterestingProperties inProps = - this.partialSolution.getInterestingProperties().clone(); - inProps.addGlobalProperties(new RequestedGlobalProperties()); - inProps.addLocalProperties(new RequestedLocalProperties()); - this.inConn.setInterestingProperties(inProps); - } - - @Override - public void clearInterestingProperties() { - super.clearInterestingProperties(); - - this.singleRoot.accept(InterestingPropertiesClearer.INSTANCE); - this.rootConnection.clearInterestingProperties(); - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - // the resulting branches are those of the step function - // because the BulkPartialSolution takes the input's branches - addClosedBranches(getSingleRootOfStepFunction().closedBranchingNodes); - List result = getSingleRootOfStepFunction().openBranches; - - this.openBranches = - (result == null || result.isEmpty()) - ? Collections.emptyList() - : result; - } - - @SuppressWarnings("unchecked") - @Override - protected void instantiateCandidate( - OperatorDescriptorSingle dps, - Channel in, - List> broadcastPlanChannels, - List target, - CostEstimator estimator, - RequestedGlobalProperties globPropsReq, - RequestedLocalProperties locPropsReq) { - // NOTES ON THE ENUMERATION OF THE STEP FUNCTION PLANS: - // Whenever we instantiate the iteration, we enumerate new candidates for the step function. - // That way, we make sure we have an appropriate plan for each candidate for the initial - // partial solution, - // we have a fitting candidate for the step function (often, work is pushed out of the step - // function). - // Among the candidates of the step function, we keep only those that meet the requested - // properties of the - // current candidate initial partial solution. That makes sure these properties exist at the - // beginning of - // the successive iteration. - - // 1) Because we enumerate multiple times, we may need to clean the cached plans - // before starting another enumeration - this.nextPartialSolution.accept(PlanCacheCleaner.INSTANCE); - if (this.terminationCriterion != null) { - this.terminationCriterion.accept(PlanCacheCleaner.INSTANCE); - } - - // 2) Give the partial solution the properties of the current candidate for the initial - // partial solution - this.partialSolution.setCandidateProperties( - in.getGlobalProperties(), in.getLocalProperties(), in); - final BulkPartialSolutionPlanNode pspn = - this.partialSolution.getCurrentPartialSolutionPlanNode(); - - // 3) Get the alternative plans - List candidates = this.nextPartialSolution.getAlternativePlans(estimator); - - // 4) Make sure that the beginning of the step function does not assume properties that - // are not also produced by the end of the step function. - - { - List newCandidates = new ArrayList(); - - for (Iterator planDeleter = candidates.iterator(); planDeleter.hasNext(); ) { - PlanNode candidate = planDeleter.next(); - - GlobalProperties atEndGlobal = candidate.getGlobalProperties(); - LocalProperties atEndLocal = candidate.getLocalProperties(); - - FeedbackPropertiesMeetRequirementsReport report = - candidate.checkPartialSolutionPropertiesMet(pspn, atEndGlobal, atEndLocal); - if (report == FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION) { - // depends only through broadcast variable on the partial solution - } else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) { - // attach a no-op node through which we create the properties of the original - // input - Channel toNoOp = new Channel(candidate); - globPropsReq.parameterizeChannel( - toNoOp, false, rootConnection.getDataExchangeMode(), false); - locPropsReq.parameterizeChannel(toNoOp); - - NoOpUnaryUdfOp noOpUnaryUdfOp = new NoOpUnaryUdfOp<>(); - noOpUnaryUdfOp.setInput(candidate.getProgramOperator()); - UnaryOperatorNode rebuildPropertiesNode = - new UnaryOperatorNode( - "Rebuild Partial Solution Properties", noOpUnaryUdfOp, true); - rebuildPropertiesNode.setParallelism(candidate.getParallelism()); - - SingleInputPlanNode rebuildPropertiesPlanNode = - new SingleInputPlanNode( - rebuildPropertiesNode, - "Rebuild Partial Solution Properties", - toNoOp, - DriverStrategy.UNARY_NO_OP); - rebuildPropertiesPlanNode.initProperties( - toNoOp.getGlobalProperties(), toNoOp.getLocalProperties()); - estimator.costOperator(rebuildPropertiesPlanNode); - - GlobalProperties atEndGlobalModified = - rebuildPropertiesPlanNode.getGlobalProperties(); - LocalProperties atEndLocalModified = - rebuildPropertiesPlanNode.getLocalProperties(); - - if (!(atEndGlobalModified.equals(atEndGlobal) - && atEndLocalModified.equals(atEndLocal))) { - FeedbackPropertiesMeetRequirementsReport report2 = - candidate.checkPartialSolutionPropertiesMet( - pspn, atEndGlobalModified, atEndLocalModified); - - if (report2 != FeedbackPropertiesMeetRequirementsReport.NOT_MET) { - newCandidates.add(rebuildPropertiesPlanNode); - } - } - - planDeleter.remove(); - } - } - - candidates.addAll(newCandidates); - } - - if (candidates.isEmpty()) { - return; - } - - // 5) Create a candidate for the Iteration Node for every remaining plan of the step - // function. - if (terminationCriterion == null) { - for (PlanNode candidate : candidates) { - BulkIterationPlanNode node = - new BulkIterationPlanNode( - this, this.getOperator().getName(), in, pspn, candidate); - GlobalProperties gProps = candidate.getGlobalProperties().clone(); - LocalProperties lProps = candidate.getLocalProperties().clone(); - node.initProperties(gProps, lProps); - target.add(node); - } - } else if (candidates.size() > 0) { - List terminationCriterionCandidates = - this.terminationCriterion.getAlternativePlans(estimator); - - SingleRootJoiner singleRoot = (SingleRootJoiner) this.singleRoot; - - for (PlanNode candidate : candidates) { - for (PlanNode terminationCandidate : terminationCriterionCandidates) { - if (singleRoot.areBranchCompatible(candidate, terminationCandidate)) { - BulkIterationPlanNode node = - new BulkIterationPlanNode( - this, - "BulkIteration (" + this.getOperator().getName() + ")", - in, - pspn, - candidate, - terminationCandidate); - GlobalProperties gProps = candidate.getGlobalProperties().clone(); - LocalProperties lProps = candidate.getLocalProperties().clone(); - node.initProperties(gProps, lProps); - target.add(node); - } - } - } - } - } - - // -------------------------------------------------------------------------------------------- - // Iteration Specific Traversals - // -------------------------------------------------------------------------------------------- - - public void acceptForStepFunction(Visitor visitor) { - this.singleRoot.accept(visitor); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkPartialSolutionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkPartialSolutionNode.java deleted file mode 100644 index 4de84eb7ec3a8..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkPartialSolutionNode.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.optimizer.dag; - -import org.apache.flink.api.common.operators.base.BulkIterationBase.PartialSolutionPlaceHolder; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.PlanNode; - -import java.util.Collections; -import java.util.List; - -/** - * The optimizer's internal representation of the partial solution that is input to a bulk - * iteration. - */ -public class BulkPartialSolutionNode extends AbstractPartialSolutionNode { - - private final BulkIterationNode iterationNode; - - public BulkPartialSolutionNode( - PartialSolutionPlaceHolder psph, BulkIterationNode iterationNode) { - super(psph); - this.iterationNode = iterationNode; - } - - // -------------------------------------------------------------------------------------------- - - public void setCandidateProperties( - GlobalProperties gProps, LocalProperties lProps, Channel initialInput) { - if (this.cachedPlans != null) { - throw new IllegalStateException(); - } else { - this.cachedPlans = - Collections.singletonList( - new BulkPartialSolutionPlanNode( - this, - "PartialSolution (" + this.getOperator().getName() + ")", - gProps, - lProps, - initialInput)); - } - } - - public BulkPartialSolutionPlanNode getCurrentPartialSolutionPlanNode() { - if (this.cachedPlans != null) { - return (BulkPartialSolutionPlanNode) this.cachedPlans.get(0); - } else { - throw new IllegalStateException(); - } - } - - public BulkIterationNode getIterationNode() { - return this.iterationNode; - } - - @Override - public void computeOutputEstimates(DataStatistics statistics) { - copyEstimates(this.iterationNode.getPredecessorNode()); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the operator (here the {@link PartialSolutionPlaceHolder}) that is represented by this - * optimizer node. - * - * @return The operator represented by this optimizer node. - */ - @Override - public PartialSolutionPlaceHolder getOperator() { - return (PartialSolutionPlaceHolder) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Bulk Partial Solution"; - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - OptimizerNode inputToIteration = this.iterationNode.getPredecessorNode(); - - addClosedBranches(inputToIteration.closedBranchingNodes); - List fromInput = - inputToIteration.getBranchesForParent(this.iterationNode.getIncomingConnection()); - this.openBranches = - (fromInput == null || fromInput.isEmpty()) - ? Collections.emptyList() - : fromInput; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CoGroupNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CoGroupNode.java deleted file mode 100644 index 1578482b14612..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CoGroupNode.java +++ /dev/null @@ -1,146 +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.optimizer.dag; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.DualInputOperator; -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.CoGroupDescriptor; -import org.apache.flink.optimizer.operators.CoGroupWithSolutionSetFirstDescriptor; -import org.apache.flink.optimizer.operators.CoGroupWithSolutionSetSecondDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; - -import java.util.Collections; -import java.util.List; - -/** The Optimizer representation of a CoGroup operator. */ -public class CoGroupNode extends TwoInputNode { - - private List dataProperties; - - public CoGroupNode(CoGroupOperatorBase operator) { - super(operator); - this.dataProperties = initializeDataProperties(operator.getCustomPartitioner()); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the operator for this CoGroup node. - * - * @return The CoGroup operator. - */ - @Override - public CoGroupOperatorBase getOperator() { - return (CoGroupOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "CoGroup"; - } - - @Override - protected List getPossibleProperties() { - return this.dataProperties; - } - - public void makeCoGroupWithSolutionSet(int solutionsetInputIndex) { - OperatorDescriptorDual op; - if (solutionsetInputIndex == 0) { - op = new CoGroupWithSolutionSetFirstDescriptor(keys1, keys2); - } else if (solutionsetInputIndex == 1) { - op = new CoGroupWithSolutionSetSecondDescriptor(keys1, keys2); - } else { - throw new IllegalArgumentException(); - } - this.dataProperties = Collections.singletonList(op); - } - - @Override - protected SemanticProperties getSemanticPropertiesForLocalPropertyFiltering() { - - // Local properties for CoGroup may only be preserved on key fields. - DualInputSemanticProperties origProps = - ((DualInputOperator) getOperator()).getSemanticProperties(); - - DualInputSemanticProperties filteredProps = new DualInputSemanticProperties(); - FieldSet readSet1 = origProps.getReadFields(0); - FieldSet readSet2 = origProps.getReadFields(1); - if (readSet1 != null) { - filteredProps.addReadFields(0, readSet1); - } - if (readSet2 != null) { - filteredProps.addReadFields(1, readSet2); - } - - // preserve only key fields (first input) - for (int f : this.keys1) { - FieldSet targets = origProps.getForwardingTargetFields(0, f); - for (int t : targets) { - filteredProps.addForwardedField(0, f, t); - } - } - - // preserve only key fields (second input) - for (int f : this.keys2) { - FieldSet targets = origProps.getForwardingTargetFields(1, f); - for (int t : targets) { - filteredProps.addForwardedField(1, f, t); - } - } - - return filteredProps; - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // for CoGroup, we currently make no reasonable default estimates - } - - private List initializeDataProperties( - Partitioner customPartitioner) { - Ordering groupOrder1 = null; - Ordering groupOrder2 = null; - - CoGroupOperatorBase cgc = getOperator(); - groupOrder1 = cgc.getGroupOrderForInputOne(); - groupOrder2 = cgc.getGroupOrderForInputTwo(); - - if (groupOrder1 != null && groupOrder1.getNumberOfFields() == 0) { - groupOrder1 = null; - } - if (groupOrder2 != null && groupOrder2.getNumberOfFields() == 0) { - groupOrder2 = null; - } - - CoGroupDescriptor descr = - new CoGroupDescriptor(this.keys1, this.keys2, groupOrder1, groupOrder2); - if (customPartitioner != null) { - descr.setCustomPartitioner(customPartitioner); - } - - return Collections.singletonList(descr); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CoGroupRawNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CoGroupRawNode.java deleted file mode 100644 index ded68052bc492..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CoGroupRawNode.java +++ /dev/null @@ -1,82 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.base.CoGroupRawOperatorBase; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.CoGroupRawDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; - -import java.util.Collections; -import java.util.List; - -/** The Optimizer representation of a CoGroupRaw operator. */ -public class CoGroupRawNode extends TwoInputNode { - private List dataProperties; - - public CoGroupRawNode(CoGroupRawOperatorBase pactContract) { - super(pactContract); - this.dataProperties = initializeDataProperties(); - } - - // -------------------------------------------------------------------------------------------- - /** - * Gets the operator for this CoGroup node. - * - * @return The CoGroup operator. - */ - @Override - public CoGroupRawOperatorBase getOperator() { - return (CoGroupRawOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "CoGroup"; - } - - @Override - protected List getPossibleProperties() { - return this.dataProperties; - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // for CoGroup, we currently make no reasonable default estimates - } - - private List initializeDataProperties() { - Ordering groupOrder1 = null; - Ordering groupOrder2 = null; - - CoGroupRawOperatorBase cgc = getOperator(); - groupOrder1 = cgc.getGroupOrderForInputOne(); - groupOrder2 = cgc.getGroupOrderForInputTwo(); - - if (groupOrder1 != null && groupOrder1.getNumberOfFields() == 0) { - groupOrder1 = null; - } - if (groupOrder2 != null && groupOrder2.getNumberOfFields() == 0) { - groupOrder2 = null; - } - - return Collections.singletonList( - new CoGroupRawDescriptor(this.keys1, this.keys2, groupOrder1, groupOrder2)); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CrossNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CrossNode.java deleted file mode 100644 index 27a9dc555c8c7..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/CrossNode.java +++ /dev/null @@ -1,139 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.base.CrossOperatorBase; -import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.operators.CrossBlockOuterFirstDescriptor; -import org.apache.flink.optimizer.operators.CrossBlockOuterSecondDescriptor; -import org.apache.flink.optimizer.operators.CrossStreamOuterFirstDescriptor; -import org.apache.flink.optimizer.operators.CrossStreamOuterSecondDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** The Optimizer representation of a Cross (Cartesian product) operator. */ -public class CrossNode extends TwoInputNode { - - private final List dataProperties; - - /** - * Creates a new CrossNode for the given operator. - * - * @param operation The Cross operator object. - */ - public CrossNode(CrossOperatorBase operation) { - super(operation); - - Configuration conf = operation.getParameters(); - String localStrategy = conf.getString(Optimizer.HINT_LOCAL_STRATEGY, null); - - CrossHint hint = operation.getCrossHint(); - - if (localStrategy != null) { - - final boolean allowBCfirst = hint != CrossHint.SECOND_IS_SMALL; - final boolean allowBCsecond = hint != CrossHint.FIRST_IS_SMALL; - - final OperatorDescriptorDual fixedDriverStrat; - if (Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST.equals( - localStrategy)) { - fixedDriverStrat = new CrossBlockOuterFirstDescriptor(allowBCfirst, allowBCsecond); - } else if (Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND.equals( - localStrategy)) { - fixedDriverStrat = new CrossBlockOuterSecondDescriptor(allowBCfirst, allowBCsecond); - } else if (Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST.equals( - localStrategy)) { - fixedDriverStrat = new CrossStreamOuterFirstDescriptor(allowBCfirst, allowBCsecond); - } else if (Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND.equals( - localStrategy)) { - fixedDriverStrat = - new CrossStreamOuterSecondDescriptor(allowBCfirst, allowBCsecond); - } else { - throw new CompilerException( - "Invalid local strategy hint for cross contract: " + localStrategy); - } - - this.dataProperties = Collections.singletonList(fixedDriverStrat); - } else if (hint == CrossHint.SECOND_IS_SMALL) { - ArrayList list = new ArrayList(); - list.add(new CrossBlockOuterSecondDescriptor(false, true)); - list.add(new CrossStreamOuterFirstDescriptor(false, true)); - this.dataProperties = list; - } else if (hint == CrossHint.FIRST_IS_SMALL) { - ArrayList list = new ArrayList(); - list.add(new CrossBlockOuterFirstDescriptor(true, false)); - list.add(new CrossStreamOuterSecondDescriptor(true, false)); - this.dataProperties = list; - } else { - ArrayList list = new ArrayList(); - list.add(new CrossBlockOuterFirstDescriptor()); - list.add(new CrossBlockOuterSecondDescriptor()); - list.add(new CrossStreamOuterFirstDescriptor()); - list.add(new CrossStreamOuterSecondDescriptor()); - this.dataProperties = list; - } - } - - // ------------------------------------------------------------------------ - - @Override - public CrossOperatorBase getOperator() { - return (CrossOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Cross"; - } - - @Override - protected List getPossibleProperties() { - return this.dataProperties; - } - - /** - * We assume that the cardinality is the product of the input cardinalities and that the result - * width is the sum of the input widths. - * - * @param statistics The statistics object to optionally access. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - long card1 = getFirstPredecessorNode().getEstimatedNumRecords(); - long card2 = getSecondPredecessorNode().getEstimatedNumRecords(); - this.estimatedNumRecords = (card1 < 0 || card2 < 0) ? -1 : card1 * card2; - - if (this.estimatedNumRecords >= 0) { - float width1 = getFirstPredecessorNode().getEstimatedAvgWidthPerOutputRecord(); - float width2 = getSecondPredecessorNode().getEstimatedAvgWidthPerOutputRecord(); - float width = (width1 <= 0 || width2 <= 0) ? -1 : width1 + width2; - - if (width > 0) { - this.estimatedOutputSize = (long) (width * this.estimatedNumRecords); - } - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DagConnection.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DagConnection.java deleted file mode 100644 index c80c4b6f4d844..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DagConnection.java +++ /dev/null @@ -1,290 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.plandump.DumpableConnection; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -/** - * A connection between two operators. Represents an intermediate result and a data exchange between - * the two operators. - * - *

The data exchange has a mode in which it performs (batch / pipelined). - * - *

The data exchange strategy may be set on this connection, in which case it is fixed and will - * not be determined during candidate plan enumeration. - * - *

During the enumeration of interesting properties, this connection also holds all interesting - * properties generated by the successor operator. - */ -public class DagConnection implements EstimateProvider, DumpableConnection { - - private final OptimizerNode source; // The source node of the connection - - private final OptimizerNode target; // The target node of the connection. - - private final ExecutionMode - dataExchangeMode; // defines whether to use batch or pipelined data exchange - - private InterestingProperties - interestingProps; // local properties that succeeding nodes are interested in - - private ShipStrategyType shipStrategy; // The data shipping strategy, if predefined. - - private TempMode materializationMode = TempMode.NONE; // the materialization mode - - private int maxDepth = -1; - - private boolean - breakPipeline; // whether this connection should break the pipeline due to potential - // deadlocks - - /** - * Creates a new Connection between two nodes. The shipping strategy is by default - * null. - * - * @param source The source node. - * @param target The target node. - * @param exchangeMode The data exchange mode (pipelined / batch / batch only for shuffles / ... - * ) - */ - public DagConnection(OptimizerNode source, OptimizerNode target, ExecutionMode exchangeMode) { - this(source, target, null, exchangeMode); - } - - /** - * Creates a new Connection between two nodes. - * - * @param source The source node. - * @param target The target node. - * @param shipStrategy The shipping strategy. - * @param exchangeMode The data exchange mode (pipelined / batch / batch only for shuffles / ... - * ) - */ - public DagConnection( - OptimizerNode source, - OptimizerNode target, - ShipStrategyType shipStrategy, - ExecutionMode exchangeMode) { - if (source == null || target == null) { - throw new NullPointerException("Source and target must not be null."); - } - this.source = source; - this.target = target; - this.shipStrategy = shipStrategy; - this.dataExchangeMode = exchangeMode; - } - - /** - * Constructor to create a result from an operator that is not consumed by another operator. - * - * @param source The source node. - * @param exchangeMode The data exchange mode (pipelined / batch / batch only for shuffles / ... - * ) - */ - public DagConnection(OptimizerNode source, ExecutionMode exchangeMode) { - if (source == null) { - throw new NullPointerException("Source must not be null."); - } - this.source = source; - this.target = null; - this.shipStrategy = ShipStrategyType.NONE; - this.dataExchangeMode = exchangeMode; - } - - /** - * Gets the source of the connection. - * - * @return The source Node. - */ - public OptimizerNode getSource() { - return this.source; - } - - /** - * Gets the target of the connection. - * - * @return The target node. - */ - public OptimizerNode getTarget() { - return this.target; - } - - /** - * Gets the shipping strategy for this connection. - * - * @return The connection's shipping strategy. - */ - public ShipStrategyType getShipStrategy() { - return this.shipStrategy; - } - - /** - * Sets the shipping strategy for this connection. - * - * @param strategy The shipping strategy to be applied to this connection. - */ - public void setShipStrategy(ShipStrategyType strategy) { - this.shipStrategy = strategy; - } - - /** - * Gets the data exchange mode to use for this connection. - * - * @return The data exchange mode to use for this connection. - */ - public ExecutionMode getDataExchangeMode() { - if (dataExchangeMode == null) { - throw new IllegalStateException( - "This connection does not have the data exchange mode set"); - } - return dataExchangeMode; - } - - /** - * Marks that this connection should do a decoupled data exchange (such as batched) rather then - * pipeline data. Connections are marked as pipeline breakers to avoid deadlock situations. - */ - public void markBreaksPipeline() { - this.breakPipeline = true; - } - - /** - * Checks whether this connection is marked to break the pipeline. - * - * @return True, if this connection is marked to break the pipeline, false otherwise. - */ - public boolean isBreakingPipeline() { - return this.breakPipeline; - } - - /** - * Gets the interesting properties object for this pact connection. If the interesting - * properties for this connections have not yet been set, this method returns null. - * - * @return The collection of all interesting properties, or null, if they have not yet been set. - */ - public InterestingProperties getInterestingProperties() { - return this.interestingProps; - } - - /** - * Sets the interesting properties for this pact connection. - * - * @param props The interesting properties. - */ - public void setInterestingProperties(InterestingProperties props) { - if (this.interestingProps == null) { - this.interestingProps = props; - } else { - throw new IllegalStateException("Interesting Properties have already been set."); - } - } - - public void clearInterestingProperties() { - this.interestingProps = null; - } - - public void initMaxDepth() { - - if (this.maxDepth == -1) { - this.maxDepth = this.source.getMaxDepth() + 1; - } else { - throw new IllegalStateException("Maximum path depth has already been initialized."); - } - } - - public int getMaxDepth() { - if (this.maxDepth != -1) { - return this.maxDepth; - } else { - throw new IllegalStateException("Maximum path depth has not been initialized."); - } - } - - // -------------------------------------------------------------------------------------------- - // Estimates - // -------------------------------------------------------------------------------------------- - - @Override - public long getEstimatedOutputSize() { - return this.source.getEstimatedOutputSize(); - } - - @Override - public long getEstimatedNumRecords() { - return this.source.getEstimatedNumRecords(); - } - - @Override - public float getEstimatedAvgWidthPerOutputRecord() { - return this.source.getEstimatedAvgWidthPerOutputRecord(); - } - - // -------------------------------------------------------------------------------------------- - - public TempMode getMaterializationMode() { - return this.materializationMode; - } - - public void setMaterializationMode(TempMode materializationMode) { - this.materializationMode = materializationMode; - } - - public boolean isOnDynamicPath() { - return this.source.isOnDynamicPath(); - } - - public int getCostWeight() { - return this.source.getCostWeight(); - } - - // -------------------------------------------------------------------------------------------- - - public String toString() { - StringBuilder buf = new StringBuilder(50); - buf.append("Connection: "); - - if (this.source == null) { - buf.append("null"); - } else { - buf.append(this.source.getOperator().getName()); - buf.append('(').append(this.source.getOperatorName()).append(')'); - } - - buf.append(" -> "); - - if (this.shipStrategy != null) { - buf.append('['); - buf.append(this.shipStrategy.name()); - buf.append(']').append(' '); - } - - if (this.target == null) { - buf.append("null"); - } else { - buf.append(this.target.getOperator().getName()); - buf.append('(').append(this.target.getOperatorName()).append(')'); - } - - return buf.toString(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java deleted file mode 100644 index abffa7b4d11b2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSinkNode.java +++ /dev/null @@ -1,257 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** The Optimizer representation of a data sink. */ -public class DataSinkNode extends OptimizerNode { - - protected DagConnection input; // The input edge - - /** - * Creates a new DataSinkNode for the given sink operator. - * - * @param sink The data sink contract object. - */ - public DataSinkNode(GenericDataSinkBase sink) { - super(sink); - } - - // -------------------------------------------------------------------------------------- - - /** - * Gets the input of the sink. - * - * @return The input connection. - */ - public DagConnection getInputConnection() { - return this.input; - } - - /** - * Gets the predecessor of this node. - * - * @return The predecessor, or null, if no predecessor has been set. - */ - public OptimizerNode getPredecessorNode() { - if (this.input != null) { - return input.getSource(); - } else { - return null; - } - } - - /** - * Gets the operator for which this optimizer sink node was created. - * - * @return The node's underlying operator. - */ - @Override - public GenericDataSinkBase getOperator() { - return (GenericDataSinkBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Data Sink"; - } - - @Override - public List getIncomingConnections() { - return Collections.singletonList(this.input); - } - - /** - * Gets all outgoing connections, which is an empty set for the data sink. - * - * @return An empty list. - */ - @Override - public List getOutgoingConnections() { - return Collections.emptyList(); - } - - @Override - public void setInput( - Map, OptimizerNode> contractToNode, ExecutionMode defaultExchangeMode) { - Operator children = getOperator().getInput(); - - final OptimizerNode pred; - final DagConnection conn; - - pred = contractToNode.get(children); - conn = new DagConnection(pred, this, defaultExchangeMode); - - // create the connection and add it - this.input = conn; - pred.addOutgoingConnection(conn); - } - - /** - * Computes the estimated outputs for the data sink. Since the sink does not modify anything, it - * simply copies the output estimates from its direct predecessor. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords(); - this.estimatedOutputSize = getPredecessorNode().getEstimatedOutputSize(); - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - final InterestingProperties iProps = new InterestingProperties(); - - { - final RequestedGlobalProperties partitioningProps = new RequestedGlobalProperties(); - iProps.addGlobalProperties(partitioningProps); - } - - { - final Ordering localOrder = getOperator().getLocalOrder(); - final RequestedLocalProperties orderProps = new RequestedLocalProperties(); - if (localOrder != null) { - orderProps.setOrdering(localOrder); - } - iProps.addLocalProperties(orderProps); - } - - this.input.setInterestingProperties(iProps); - } - - // -------------------------------------------------------------------------------------------- - // Branch Handling - // -------------------------------------------------------------------------------------------- - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - // we need to track open branches even in the sinks, because they get "closed" when - // we build a single "root" for the data flow plan - addClosedBranches(getPredecessorNode().closedBranchingNodes); - this.openBranches = getPredecessorNode().getBranchesForParent(this.input); - } - - @Override - protected List getBranchesForParent(DagConnection parent) { - // return our own stack of open branches, because nothing is added - return this.openBranches; - } - - // -------------------------------------------------------------------------------------------- - // Recursive Optimization - // -------------------------------------------------------------------------------------------- - - @Override - public List getAlternativePlans(CostEstimator estimator) { - // check if we have a cached version - if (this.cachedPlans != null) { - return this.cachedPlans; - } - - // calculate alternative sub-plans for predecessor - List subPlans = getPredecessorNode().getAlternativePlans(estimator); - List outputPlans = new ArrayList(); - - final int parallelism = getParallelism(); - final int inDop = getPredecessorNode().getParallelism(); - - final ExecutionMode executionMode = this.input.getDataExchangeMode(); - final boolean dopChange = parallelism != inDop; - final boolean breakPipeline = this.input.isBreakingPipeline(); - - InterestingProperties ips = this.input.getInterestingProperties(); - for (PlanNode p : subPlans) { - for (RequestedGlobalProperties gp : ips.getGlobalProperties()) { - for (RequestedLocalProperties lp : ips.getLocalProperties()) { - Channel c = new Channel(p); - gp.parameterizeChannel(c, dopChange, executionMode, breakPipeline); - lp.parameterizeChannel(c); - c.setRequiredLocalProps(lp); - c.setRequiredGlobalProps(gp); - - // no need to check whether the created properties meet what we need in case - // of ordering or global ordering, because the only interesting properties we - // have - // are what we require - outputPlans.add( - new SinkPlanNode( - this, "DataSink (" + this.getOperator().getName() + ")", c)); - } - } - } - - // cost and prune the plans - for (PlanNode node : outputPlans) { - estimator.costOperator(node); - } - prunePlanAlternatives(outputPlans); - - this.cachedPlans = outputPlans; - return outputPlans; - } - - // -------------------------------------------------------------------------------------------- - // Function Annotation Handling - // -------------------------------------------------------------------------------------------- - - @Override - public SemanticProperties getSemanticProperties() { - return new EmptySemanticProperties(); - } - - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - if (getPredecessorNode() != null) { - getPredecessorNode().accept(visitor); - } else { - throw new CompilerException(); - } - visitor.postVisit(this); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java deleted file mode 100644 index 26fe2776a4dfb..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/DataSourceNode.java +++ /dev/null @@ -1,315 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.io.FileInputFormat; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.io.NonParallelInput; -import org.apache.flink.api.common.io.ReplicatingInputFormat; -import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.GenericDataSourceBase.SplitDataProperties; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** The optimizer's internal representation of a data source. */ -public class DataSourceNode extends OptimizerNode { - - private final boolean sequentialInput; - - private final boolean replicatedInput; - - private GlobalProperties gprops; - - private LocalProperties lprops; - - /** - * Creates a new DataSourceNode for the given contract. - * - * @param pactContract The data source contract object. - */ - public DataSourceNode(GenericDataSourceBase pactContract) { - super(pactContract); - - if (pactContract.getUserCodeWrapper().getUserCodeClass() == null) { - throw new IllegalArgumentException("Input format has not been set."); - } - - if (NonParallelInput.class.isAssignableFrom( - pactContract.getUserCodeWrapper().getUserCodeClass())) { - setParallelism(1); - this.sequentialInput = true; - } else { - this.sequentialInput = false; - } - - this.replicatedInput = - ReplicatingInputFormat.class.isAssignableFrom( - pactContract.getUserCodeWrapper().getUserCodeClass()); - - this.gprops = new GlobalProperties(); - this.lprops = new LocalProperties(); - - SplitDataProperties splitProps = pactContract.getSplitDataProperties(); - - if (replicatedInput) { - this.gprops.setFullyReplicated(); - this.lprops = new LocalProperties(); - } else if (splitProps != null) { - // configure data properties of data source using split properties - setDataPropertiesFromSplitProperties(splitProps); - } - } - - /** - * Gets the contract object for this data source node. - * - * @return The contract. - */ - @Override - public GenericDataSourceBase getOperator() { - return (GenericDataSourceBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Data Source"; - } - - @Override - public void setParallelism(int parallelism) { - // if unsplittable, parallelism remains at 1 - if (!this.sequentialInput) { - super.setParallelism(parallelism); - } - } - - @Override - public List getIncomingConnections() { - return Collections.emptyList(); - } - - @Override - public void setInput( - Map, OptimizerNode> contractToNode, - ExecutionMode defaultDataExchangeMode) {} - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // see, if we have a statistics object that can tell us a bit about the file - if (statistics != null) { - // instantiate the input format, as this is needed by the statistics - InputFormat format; - String inFormatDescription = ""; - - try { - format = getOperator().getFormatWrapper().getUserCodeObject(); - Configuration config = getOperator().getParameters(); - format.configure(config); - } catch (Throwable t) { - if (Optimizer.LOG.isWarnEnabled()) { - Optimizer.LOG.warn( - "Could not instantiate InputFormat to obtain statistics." - + " Limited statistics will be available.", - t); - } - return; - } - try { - inFormatDescription = format.toString(); - } catch (Throwable t) { - // we can ignore this error, as it only prevents us to use a cosmetic string - } - - // first of all, get the statistics from the cache - final String statisticsKey = getOperator().getStatisticsKey(); - final BaseStatistics cachedStatistics = statistics.getBaseStatistics(statisticsKey); - - BaseStatistics bs = null; - try { - bs = format.getStatistics(cachedStatistics); - } catch (Throwable t) { - if (Optimizer.LOG.isWarnEnabled()) { - Optimizer.LOG.warn( - "Error obtaining statistics from input format: " + t.getMessage(), t); - } - } - - if (bs != null) { - final long len = bs.getTotalInputSize(); - if (len == BaseStatistics.SIZE_UNKNOWN) { - if (Optimizer.LOG.isInfoEnabled()) { - Optimizer.LOG.info( - "Compiler could not determine the size of input '" - + inFormatDescription - + "'. Using default estimates."); - } - } else if (len >= 0) { - this.estimatedOutputSize = len; - } - - final long card = bs.getNumberOfRecords(); - if (card != BaseStatistics.NUM_RECORDS_UNKNOWN) { - this.estimatedNumRecords = card; - } - } - } - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - // no children, so nothing to compute - } - - @Override - public void computeUnclosedBranchStack() { - // because there are no inputs, there are no unclosed branches. - this.openBranches = Collections.emptyList(); - } - - @Override - public List getAlternativePlans(CostEstimator estimator) { - if (this.cachedPlans != null) { - return this.cachedPlans; - } - - SourcePlanNode candidate = - new SourcePlanNode( - this, - "DataSource (" + this.getOperator().getName() + ")", - this.gprops, - this.lprops); - - if (!replicatedInput) { - candidate.updatePropertiesWithUniqueSets(getUniqueFields()); - - final Costs costs = new Costs(); - if (FileInputFormat.class.isAssignableFrom( - getOperator().getFormatWrapper().getUserCodeClass()) - && this.estimatedOutputSize >= 0) { - estimator.addFileInputCost(this.estimatedOutputSize, costs); - } - candidate.setCosts(costs); - } else { - // replicated input - final Costs costs = new Costs(); - InputFormat inputFormat = - ((ReplicatingInputFormat) - getOperator().getFormatWrapper().getUserCodeObject()) - .getReplicatedInputFormat(); - if (FileInputFormat.class.isAssignableFrom(inputFormat.getClass()) - && this.estimatedOutputSize >= 0) { - estimator.addFileInputCost(this.estimatedOutputSize * this.getParallelism(), costs); - } - candidate.setCosts(costs); - } - - // since there is only a single plan for the data-source, return a list with that element - // only - List plans = new ArrayList(1); - plans.add(candidate); - - this.cachedPlans = plans; - return plans; - } - - @Override - public SemanticProperties getSemanticProperties() { - return new EmptySemanticProperties(); - } - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - visitor.postVisit(this); - } - } - - private void setDataPropertiesFromSplitProperties(SplitDataProperties splitProps) { - - // set global properties - int[] partitionKeys = splitProps.getSplitPartitionKeys(); - Partitioner partitioner = splitProps.getSplitPartitioner(); - - if (partitionKeys != null && partitioner != null) { - this.gprops.setCustomPartitioned(new FieldList(partitionKeys), partitioner); - } else if (partitionKeys != null) { - this.gprops.setAnyPartitioning(new FieldList(partitionKeys)); - } - // set local properties - int[] groupingKeys = splitProps.getSplitGroupKeys(); - Ordering ordering = splitProps.getSplitOrder(); - - // more than one split per source tasks possible. - // adapt split grouping and sorting - if (ordering != null) { - - // sorting falls back to grouping because a source can read multiple, - // randomly assigned splits - groupingKeys = ordering.getFieldPositions(); - } - - if (groupingKeys != null && partitionKeys != null) { - // check if grouping is also valid across splits, i.e., whether grouping keys are - // valid superset of partition keys - boolean allFieldsIncluded = true; - for (int i : partitionKeys) { - boolean fieldIncluded = false; - for (int j : groupingKeys) { - if (i == j) { - fieldIncluded = true; - break; - } - } - if (!fieldIncluded) { - allFieldsIncluded = false; - break; - } - } - if (allFieldsIncluded) { - this.lprops = LocalProperties.forGrouping(new FieldList(groupingKeys)); - } else { - this.lprops = new LocalProperties(); - } - - } else { - this.lprops = new LocalProperties(); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/EstimateProvider.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/EstimateProvider.java deleted file mode 100644 index a7b9745a42d04..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/EstimateProvider.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.optimizer.dag; - -/** - * Methods for operators / connections that provide estimated about data size and characteristics. - */ -public interface EstimateProvider { - - /** - * Gets the estimated output size from this node. - * - * @return The estimated output size. - */ - long getEstimatedOutputSize(); - - /** - * Gets the estimated number of records in the output of this node. - * - * @return The estimated number of records. - */ - long getEstimatedNumRecords(); - - /** - * Gets the estimated number of bytes per record. - * - * @return The estimated number of bytes per record. - */ - float getEstimatedAvgWidthPerOutputRecord(); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/FilterNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/FilterNode.java deleted file mode 100644 index 74e69d855925d..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/FilterNode.java +++ /dev/null @@ -1,72 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.FilterOperatorBase; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.FilterDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of a Filter operator node. */ -public class FilterNode extends SingleInputNode { - - private final List possibleProperties; - - public FilterNode(FilterOperatorBase operator) { - super(operator); - this.possibleProperties = - Collections.singletonList(new FilterDescriptor()); - } - - @Override - public FilterOperatorBase getOperator() { - return (FilterOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Filter"; - } - - @Override - public SemanticProperties getSemanticProperties() { - return new SingleInputSemanticProperties.AllFieldsForwardedProperties(); - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - /** - * Computes the estimates for the Filter operator. Since it applies a filter on the data we - * assume a cardinality decrease. To give the system a hint at data decrease, we use a default - * magic number to indicate a 0.5 decrease. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedNumRecords = (long) (getPredecessorNode().getEstimatedNumRecords() * 0.5); - this.estimatedOutputSize = (long) (getPredecessorNode().getEstimatedOutputSize() * 0.5); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/FlatMapNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/FlatMapNode.java deleted file mode 100644 index ba52550ffdded..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/FlatMapNode.java +++ /dev/null @@ -1,65 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.base.FlatMapOperatorBase; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.FlatMapDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of a FlatMap operator node. */ -public class FlatMapNode extends SingleInputNode { - - private final List possibleProperties; - - public FlatMapNode(FlatMapOperatorBase operator) { - super(operator); - - this.possibleProperties = - Collections.singletonList(new FlatMapDescriptor()); - } - - @Override - public FlatMapOperatorBase getOperator() { - return (FlatMapOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "FlatMap"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - /** - * Computes the estimates for the FlatMap operator. Since it un-nests, we assume a cardinality - * increase. To give the system a hint at data increase, we take a default magic number of a 5 - * times increase. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords() * 5; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java deleted file mode 100644 index b3f0014cb6cc6..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupCombineNode.java +++ /dev/null @@ -1,127 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.AllGroupCombineProperties; -import org.apache.flink.optimizer.operators.GroupCombineProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.Collections; -import java.util.List; - -/** The optimizer representation of a GroupCombineNode operation. */ -public class GroupCombineNode extends SingleInputNode { - - private final List possibleProperties; - - /** - * Creates a new optimizer node for the given operator. - * - * @param operator The reduce operation. - */ - public GroupCombineNode(GroupCombineOperatorBase operator) { - super(operator); - - if (this.keys == null) { - // case of a key-less reducer. force a parallelism of 1 - setParallelism(1); - } - - this.possibleProperties = initPossibleProperties(); - } - - private List initPossibleProperties() { - - // check if we can work with a grouping (simple reducer), or if we need ordering because of - // a group order - Ordering groupOrder = getOperator().getGroupOrder(); - if (groupOrder != null && groupOrder.getNumberOfFields() == 0) { - groupOrder = null; - } - - OperatorDescriptorSingle props = - (this.keys == null - ? new AllGroupCombineProperties() - : new GroupCombineProperties(this.keys, groupOrder)); - - return Collections.singletonList(props); - } - - // ------------------------------------------------------------------------ - - /** - * Gets the operator represented by this optimizer node. - * - * @return The operator represented by this optimizer node. - */ - @Override - public GroupCombineOperatorBase getOperator() { - return (GroupCombineOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "GroupCombine"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - @Override - protected SemanticProperties getSemanticPropertiesForLocalPropertyFiltering() { - - // Local properties for GroupCombine may only be preserved on key fields. - SingleInputSemanticProperties origProps = - ((SingleInputOperator) getOperator()).getSemanticProperties(); - SingleInputSemanticProperties filteredProps = new SingleInputSemanticProperties(); - FieldSet readSet = origProps.getReadFields(0); - if (readSet != null) { - filteredProps.addReadFields(readSet); - } - - // only add forward field information for key fields - if (this.keys != null) { - for (int f : this.keys) { - FieldSet targets = origProps.getForwardingTargetFields(0, f); - for (int t : targets) { - filteredProps.addForwardedField(f, t); - } - } - } - return filteredProps; - } - - // -------------------------------------------------------------------------------------------- - // Estimates - // -------------------------------------------------------------------------------------------- - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // no real estimates possible for a reducer. - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java deleted file mode 100644 index 132a524fe661d..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/GroupReduceNode.java +++ /dev/null @@ -1,198 +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.optimizer.dag; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.operators.AllGroupReduceProperties; -import org.apache.flink.optimizer.operators.AllGroupWithPartialPreGroupProperties; -import org.apache.flink.optimizer.operators.GroupReduceProperties; -import org.apache.flink.optimizer.operators.GroupReduceWithCombineProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.Collections; -import java.util.List; - -/** The optimizer representation of a GroupReduce operation. */ -public class GroupReduceNode extends SingleInputNode { - - private final List possibleProperties; - - private final String operatorName; - - private GroupReduceNode combinerUtilityNode; - - /** - * Creates a new optimizer node for the given operator. - * - * @param operator The reduce operation. - */ - public GroupReduceNode(GroupReduceOperatorBase operator) { - super(operator); - this.operatorName = "GroupReduce"; - - if (this.keys == null) { - // case of a key-less reducer. force a parallelism of 1 - setParallelism(1); - } - - this.possibleProperties = initPossibleProperties(operator.getCustomPartitioner()); - } - - private GroupReduceNode(GroupReduceNode reducerToCopyForCombiner) { - super(reducerToCopyForCombiner); - this.operatorName = "GroupCombine"; - - this.possibleProperties = Collections.emptyList(); - } - - private List initPossibleProperties( - Partitioner customPartitioner) { - // see if an internal hint dictates the strategy to use - final Configuration conf = getOperator().getParameters(); - final String localStrategy = conf.getString(Optimizer.HINT_LOCAL_STRATEGY, null); - - final boolean useCombiner; - if (localStrategy != null) { - if (Optimizer.HINT_LOCAL_STRATEGY_SORT.equals(localStrategy)) { - useCombiner = false; - } else if (Optimizer.HINT_LOCAL_STRATEGY_COMBINING_SORT.equals(localStrategy)) { - if (!isCombineable()) { - Optimizer.LOG.warn( - "Strategy hint for GroupReduce '" - + getOperator().getName() - + "' requires combinable reduce, but user function is not marked combinable."); - } - useCombiner = true; - } else { - throw new CompilerException( - "Invalid local strategy hint for match contract: " + localStrategy); - } - } else { - useCombiner = isCombineable(); - } - - // check if we can work with a grouping (simple reducer), or if we need ordering because of - // a group order - Ordering groupOrder = null; - if (getOperator() != null) { - groupOrder = getOperator().getGroupOrder(); - if (groupOrder != null && groupOrder.getNumberOfFields() == 0) { - groupOrder = null; - } - } - - OperatorDescriptorSingle props = - useCombiner - ? (this.keys == null - ? new AllGroupWithPartialPreGroupProperties() - : new GroupReduceWithCombineProperties( - this.keys, groupOrder, customPartitioner)) - : (this.keys == null - ? new AllGroupReduceProperties() - : new GroupReduceProperties( - this.keys, groupOrder, customPartitioner)); - - return Collections.singletonList(props); - } - - // ------------------------------------------------------------------------ - - /** - * Gets the operator represented by this optimizer node. - * - * @return The operator represented by this optimizer node. - */ - @Override - public GroupReduceOperatorBase getOperator() { - return (GroupReduceOperatorBase) super.getOperator(); - } - - /** - * Checks, whether a combiner function has been given for the function encapsulated by this - * reduce contract. - * - * @return True, if a combiner has been given, false otherwise. - */ - public boolean isCombineable() { - return getOperator().isCombinable(); - } - - @Override - public String getOperatorName() { - return this.operatorName; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - @Override - protected SemanticProperties getSemanticPropertiesForLocalPropertyFiltering() { - // Local properties for GroupReduce may only be preserved on key fields. - SingleInputSemanticProperties origProps = getOperator().getSemanticProperties(); - SingleInputSemanticProperties filteredProps = new SingleInputSemanticProperties(); - FieldSet readSet = origProps.getReadFields(0); - if (readSet != null) { - filteredProps.addReadFields(readSet); - } - - // only add forward field information for key fields - if (this.keys != null) { - for (int f : this.keys) { - FieldSet targets = origProps.getForwardingTargetFields(0, f); - for (int t : targets) { - filteredProps.addForwardedField(f, t); - } - } - } - return filteredProps; - } - - // -------------------------------------------------------------------------------------------- - // Estimates - // -------------------------------------------------------------------------------------------- - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // no real estimates possible for a reducer. - } - - public GroupReduceNode getCombinerUtilityNode() { - if (this.combinerUtilityNode == null) { - this.combinerUtilityNode = new GroupReduceNode(this); - - // we conservatively assume the combiner returns the same data size as it consumes - this.combinerUtilityNode.estimatedOutputSize = - getPredecessorNode().getEstimatedOutputSize(); - this.combinerUtilityNode.estimatedNumRecords = - getPredecessorNode().getEstimatedNumRecords(); - } - return this.combinerUtilityNode; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/InterestingPropertiesClearer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/InterestingPropertiesClearer.java deleted file mode 100644 index 3b6b0c92b58df..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/InterestingPropertiesClearer.java +++ /dev/null @@ -1,39 +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.optimizer.dag; - -import org.apache.flink.util.Visitor; - -final class InterestingPropertiesClearer implements Visitor { - - static final InterestingPropertiesClearer INSTANCE = new InterestingPropertiesClearer(); - - @Override - public boolean preVisit(OptimizerNode visitable) { - if (visitable.getInterestingProperties() != null) { - visitable.clearInterestingProperties(); - return true; - } else { - return false; - } - } - - @Override - public void postVisit(OptimizerNode visitable) {} -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/IterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/IterationNode.java deleted file mode 100644 index 2dc5bf0b2f284..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/IterationNode.java +++ /dev/null @@ -1,27 +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.optimizer.dag; - -import org.apache.flink.util.Visitor; - -/** */ -public interface IterationNode { - - void acceptForStepFunction(Visitor visitor); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java deleted file mode 100644 index 6e840f0de6de4..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/JoinNode.java +++ /dev/null @@ -1,195 +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.optimizer.dag; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.operators.AbstractJoinDescriptor; -import org.apache.flink.optimizer.operators.HashJoinBuildFirstProperties; -import org.apache.flink.optimizer.operators.HashJoinBuildSecondProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; -import org.apache.flink.optimizer.operators.SortMergeInnerJoinDescriptor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** The Optimizer representation of a join operator. */ -public class JoinNode extends TwoInputNode { - - private List dataProperties; - - /** - * Creates a new JoinNode for the given join operator. - * - * @param joinOperatorBase The join operator object. - */ - public JoinNode(InnerJoinOperatorBase joinOperatorBase) { - super(joinOperatorBase); - - this.dataProperties = - getDataProperties( - joinOperatorBase, - joinOperatorBase.getJoinHint(), - joinOperatorBase.getCustomPartitioner()); - } - - // ------------------------------------------------------------------------ - - /** - * Gets the contract object for this match node. - * - * @return The contract. - */ - @Override - public InnerJoinOperatorBase getOperator() { - return (InnerJoinOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Join"; - } - - @Override - protected List getPossibleProperties() { - return this.dataProperties; - } - - public void makeJoinWithSolutionSet(int solutionsetInputIndex) { - OperatorDescriptorDual op; - if (solutionsetInputIndex == 0) { - op = new HashJoinBuildFirstProperties(this.keys1, this.keys2); - } else if (solutionsetInputIndex == 1) { - op = new HashJoinBuildSecondProperties(this.keys1, this.keys2); - } else { - throw new IllegalArgumentException(); - } - - this.dataProperties = Collections.singletonList(op); - } - - /** - * The default estimates build on the principle of inclusion: The smaller input key domain is - * included in the larger input key domain. We also assume that every key from the larger input - * has one join partner in the smaller input. The result cardinality is hence the larger one. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - long card1 = getFirstPredecessorNode().getEstimatedNumRecords(); - long card2 = getSecondPredecessorNode().getEstimatedNumRecords(); - this.estimatedNumRecords = (card1 < 0 || card2 < 0) ? -1 : Math.max(card1, card2); - - if (this.estimatedNumRecords >= 0) { - float width1 = getFirstPredecessorNode().getEstimatedAvgWidthPerOutputRecord(); - float width2 = getSecondPredecessorNode().getEstimatedAvgWidthPerOutputRecord(); - float width = (width1 <= 0 || width2 <= 0) ? -1 : width1 + width2; - - if (width > 0) { - this.estimatedOutputSize = (long) (width * this.estimatedNumRecords); - } - } - } - - private List getDataProperties( - InnerJoinOperatorBase joinOperatorBase, - JoinHint joinHint, - Partitioner customPartitioner) { - // see if an internal hint dictates the strategy to use - Configuration conf = joinOperatorBase.getParameters(); - String localStrategy = conf.getString(Optimizer.HINT_LOCAL_STRATEGY, null); - - if (localStrategy != null) { - final AbstractJoinDescriptor fixedDriverStrat; - if (Optimizer.HINT_LOCAL_STRATEGY_SORT_BOTH_MERGE.equals(localStrategy) - || Optimizer.HINT_LOCAL_STRATEGY_SORT_FIRST_MERGE.equals(localStrategy) - || Optimizer.HINT_LOCAL_STRATEGY_SORT_SECOND_MERGE.equals(localStrategy) - || Optimizer.HINT_LOCAL_STRATEGY_MERGE.equals(localStrategy)) { - fixedDriverStrat = new SortMergeInnerJoinDescriptor(this.keys1, this.keys2); - } else if (Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST.equals(localStrategy)) { - fixedDriverStrat = new HashJoinBuildFirstProperties(this.keys1, this.keys2); - } else if (Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND.equals(localStrategy)) { - fixedDriverStrat = new HashJoinBuildSecondProperties(this.keys1, this.keys2); - } else { - throw new CompilerException( - "Invalid local strategy hint for match contract: " + localStrategy); - } - - if (customPartitioner != null) { - fixedDriverStrat.setCustomPartitioner(customPartitioner); - } - - ArrayList list = new ArrayList(); - list.add(fixedDriverStrat); - return list; - } else { - ArrayList list = new ArrayList(); - - joinHint = joinHint == null ? JoinHint.OPTIMIZER_CHOOSES : joinHint; - - switch (joinHint) { - case BROADCAST_HASH_FIRST: - list.add( - new HashJoinBuildFirstProperties( - this.keys1, this.keys2, true, false, false)); - break; - case BROADCAST_HASH_SECOND: - list.add( - new HashJoinBuildSecondProperties( - this.keys1, this.keys2, false, true, false)); - break; - case REPARTITION_HASH_FIRST: - list.add( - new HashJoinBuildFirstProperties( - this.keys1, this.keys2, false, false, true)); - break; - case REPARTITION_HASH_SECOND: - list.add( - new HashJoinBuildSecondProperties( - this.keys1, this.keys2, false, false, true)); - break; - case REPARTITION_SORT_MERGE: - list.add( - new SortMergeInnerJoinDescriptor( - this.keys1, this.keys2, false, false, true)); - break; - case OPTIMIZER_CHOOSES: - list.add(new SortMergeInnerJoinDescriptor(this.keys1, this.keys2)); - list.add(new HashJoinBuildFirstProperties(this.keys1, this.keys2)); - list.add(new HashJoinBuildSecondProperties(this.keys1, this.keys2)); - break; - default: - throw new CompilerException("Unrecognized join hint: " + joinHint); - } - - if (customPartitioner != null) { - for (OperatorDescriptorDual descr : list) { - ((AbstractJoinDescriptor) descr).setCustomPartitioner(customPartitioner); - } - } - - return list; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MapNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MapNode.java deleted file mode 100644 index 06d78924dac96..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MapNode.java +++ /dev/null @@ -1,64 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.MapDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of a Map operator node. */ -public class MapNode extends SingleInputNode { - - private final List possibleProperties; - - /** - * Creates a new MapNode for the given operator. - * - * @param operator The map operator. - */ - public MapNode(SingleInputOperator operator) { - super(operator); - - this.possibleProperties = - Collections.singletonList(new MapDescriptor()); - } - - @Override - public String getOperatorName() { - return "Map"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - /** - * Computes the estimates for the Map operator. We assume that by default, Map takes one value - * and transforms it into another value. The cardinality consequently stays the same. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MapPartitionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MapPartitionNode.java deleted file mode 100644 index a74f95a9ec2f9..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/MapPartitionNode.java +++ /dev/null @@ -1,82 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.MapPartitionDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of a MapPartition operator node. */ -public class MapPartitionNode extends SingleInputNode { - - private final List possibleProperties; - - /** - * Creates a new MapNode for the given contract. - * - * @param operator The map partition contract object. - */ - public MapPartitionNode(SingleInputOperator operator) { - super(operator); - - this.possibleProperties = - Collections.singletonList(new MapPartitionDescriptor()); - } - - @Override - public String getOperatorName() { - return "MapPartition"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - @Override - protected SemanticProperties getSemanticPropertiesForLocalPropertyFiltering() { - - // Local properties for MapPartition may not be preserved. - SingleInputSemanticProperties origProps = - ((SingleInputOperator) getOperator()).getSemanticProperties(); - SingleInputSemanticProperties filteredProps = new SingleInputSemanticProperties(); - FieldSet readSet = origProps.getReadFields(0); - if (readSet != null) { - filteredProps.addReadFields(readSet); - } - - return filteredProps; - } - - /** - * Computes the estimates for the MapPartition operator. We assume that by default, Map takes - * one value and transforms it into another value. The cardinality consequently stays the same. - */ - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // we really cannot make any estimates here - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/NoOpNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/NoOpNode.java deleted file mode 100644 index 730ed38f8abc0..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/NoOpNode.java +++ /dev/null @@ -1,41 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.NoOpDescriptor; - -/** The optimizer's internal representation of a No Operation node. */ -public class NoOpNode extends UnaryOperatorNode { - - public NoOpNode() { - super("No Op", new FieldSet(), new NoOpDescriptor()); - } - - public NoOpNode(String name) { - super(name, new FieldSet(), new NoOpDescriptor()); - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords(); - this.estimatedOutputSize = getPredecessorNode().getEstimatedOutputSize(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java deleted file mode 100644 index a8f38c9f18b72..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java +++ /dev/null @@ -1,1212 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.AbstractUdfOperator; -import org.apache.flink.api.common.operators.CompilerHints; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plandump.DumpableConnection; -import org.apache.flink.optimizer.plandump.DumpableNode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitable; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * The OptimizerNode is the base class of all nodes in the optimizer DAG. The optimizer DAG is the - * optimizer's representation of a program, created before the actual optimization (which creates - * different candidate plans and computes their cost). - * - *

Nodes in the DAG correspond (almost) one-to-one to the operators in a program. The optimizer - * DAG is constructed to hold the additional information that the optimizer needs: - * - *

    - *
  • Estimates of the data size processed by each operator - *
  • Helper structures to track where the data flow "splits" and "joins", to support flows that - * are DAGs but not trees. - *
  • Tags and weights to differentiate between loop-variant and -invariant parts of an iteration - *
  • Interesting properties to be used during the enumeration of candidate plans - *
- */ -public abstract class OptimizerNode - implements Visitable, EstimateProvider, DumpableNode { - - public static final int MAX_DYNAMIC_PATH_COST_WEIGHT = 100; - - // -------------------------------------------------------------------------------------------- - // Members - // -------------------------------------------------------------------------------------------- - - private final Operator operator; // The operator (Reduce / Join / DataSource / ...) - - private List broadcastConnectionNames = - new ArrayList(); // the broadcast inputs names of this node - - private List broadcastConnections = - new ArrayList(); // the broadcast inputs of this node - - private List outgoingConnections; // The links to succeeding nodes - - private InterestingProperties intProps; // the interesting properties of this node - - // --------------------------------- Branch Handling ------------------------------------------ - - protected List - openBranches; // stack of branches in the sub-graph that are not joined - - protected Set - closedBranchingNodes; // stack of branching nodes which have already been closed - - protected List - hereJoinedBranches; // the branching nodes (node with multiple outputs) - // that are partially joined (through multiple inputs or broadcast vars) - - // ---------------------------- Estimates and Annotations ------------------------------------- - - protected long estimatedOutputSize = -1; // the estimated size of the output (bytes) - - protected long estimatedNumRecords = - -1; // the estimated number of key/value pairs in the output - - protected Set
- uniqueFields; // set of attributes that will always be unique after this node - - // --------------------------------- General Parameters --------------------------------------- - - private int parallelism = - ExecutionConfig.PARALLELISM_DEFAULT; // the number of parallel instances of this node - - private long minimalMemoryPerSubTask = -1; - - protected int id = -1; // the id for this node. - - protected int costWeight = 1; // factor to weight the costs for dynamic paths - - protected boolean onDynamicPath; - - protected List - cachedPlans; // cache candidates, because the may be accessed repeatedly - - // ------------------------------------------------------------------------ - // Constructor / Setup - // ------------------------------------------------------------------------ - - /** - * Creates a new optimizer node that represents the given program operator. - * - * @param op The operator that the node represents. - */ - public OptimizerNode(Operator op) { - this.operator = op; - readStubAnnotations(); - } - - protected OptimizerNode(OptimizerNode toCopy) { - this.operator = toCopy.operator; - this.intProps = toCopy.intProps; - - this.openBranches = toCopy.openBranches; - this.closedBranchingNodes = toCopy.closedBranchingNodes; - - this.estimatedOutputSize = toCopy.estimatedOutputSize; - this.estimatedNumRecords = toCopy.estimatedNumRecords; - - this.parallelism = toCopy.parallelism; - this.minimalMemoryPerSubTask = toCopy.minimalMemoryPerSubTask; - - this.id = toCopy.id; - this.costWeight = toCopy.costWeight; - this.onDynamicPath = toCopy.onDynamicPath; - } - - // ------------------------------------------------------------------------ - // Methods specific to unary- / binary- / special nodes - // ------------------------------------------------------------------------ - - /** - * Gets the name of this node, which is the name of the function/operator, or data source / data - * sink. - * - * @return The node name. - */ - public abstract String getOperatorName(); - - /** - * This function connects the predecessors to this operator. - * - * @param operatorToNode The map from program operators to optimizer nodes. - * @param defaultExchangeMode The data exchange mode to use, if the operator does not specify - * one. - */ - public abstract void setInput( - Map, OptimizerNode> operatorToNode, ExecutionMode defaultExchangeMode); - - /** - * This function connects the operators that produce the broadcast inputs to this operator. - * - * @param operatorToNode The map from program operators to optimizer nodes. - * @param defaultExchangeMode The data exchange mode to use, if the operator does not specify - * one. - * @throws CompilerException - */ - public void setBroadcastInputs( - Map, OptimizerNode> operatorToNode, ExecutionMode defaultExchangeMode) { - // skip for Operators that don't support broadcast variables - if (!(getOperator() instanceof AbstractUdfOperator)) { - return; - } - - // get all broadcast inputs - AbstractUdfOperator operator = ((AbstractUdfOperator) getOperator()); - - // create connections and add them - for (Map.Entry> input : operator.getBroadcastInputs().entrySet()) { - OptimizerNode predecessor = operatorToNode.get(input.getValue()); - DagConnection connection = - new DagConnection( - predecessor, this, ShipStrategyType.BROADCAST, defaultExchangeMode); - addBroadcastConnection(input.getKey(), connection); - predecessor.addOutgoingConnection(connection); - } - } - - /** - * Gets all incoming connections of this node. This method needs to be overridden by subclasses - * to return the children. - * - * @return The list of incoming connections. - */ - public abstract List getIncomingConnections(); - - /** - * Tells the node to compute the interesting properties for its inputs. The interesting - * properties for the node itself must have been computed before. The node must then see how - * many of interesting properties it preserves and add its own. - * - * @param estimator The {@code CostEstimator} instance to use for plan cost estimation. - */ - public abstract void computeInterestingPropertiesForInputs(CostEstimator estimator); - - /** - * This method causes the node to compute the description of open branches in its sub-plan. An - * open branch describes, that a (transitive) child node had multiple outputs, which have not - * all been re-joined in the sub-plan. This method needs to set the openBranches - * field to a stack of unclosed branches, the latest one top. A branch is considered closed, if - * some later node sees all of the branching node's outputs, no matter if there have been more - * branches to different paths in the meantime. - */ - public abstract void computeUnclosedBranchStack(); - - protected List computeUnclosedBranchStackForBroadcastInputs( - List branchesSoFar) { - // handle the data flow branching for the broadcast inputs - for (DagConnection broadcastInput : getBroadcastConnections()) { - OptimizerNode bcSource = broadcastInput.getSource(); - addClosedBranches(bcSource.closedBranchingNodes); - - List bcBranches = - bcSource.getBranchesForParent(broadcastInput); - - ArrayList mergedBranches = - new ArrayList(); - mergeLists(branchesSoFar, bcBranches, mergedBranches, true); - branchesSoFar = - mergedBranches.isEmpty() - ? Collections.emptyList() - : mergedBranches; - } - - return branchesSoFar; - } - - /** - * Computes the plan alternatives for this node, an implicitly for all nodes that are children - * of this node. This method must determine for each alternative the global and local properties - * and the costs. This method may recursively call getAlternatives() on its - * children to get their plan alternatives, and build its own alternatives on top of those. - * - * @param estimator The cost estimator used to estimate the costs of each plan alternative. - * @return A list containing all plan alternatives. - */ - public abstract List getAlternativePlans(CostEstimator estimator); - - /** - * This method implements the visit of a depth-first graph traversing visitor. Implementers must - * first call the preVisit() method, then hand the visitor to their children, and - * finally call the postVisit() method. - * - * @param visitor The graph traversing visitor. - * @see org.apache.flink.util.Visitable#accept(org.apache.flink.util.Visitor) - */ - @Override - public abstract void accept(Visitor visitor); - - public abstract SemanticProperties getSemanticProperties(); - - // ------------------------------------------------------------------------ - // Getters / Setters - // ------------------------------------------------------------------------ - - @Override - public Iterable getPredecessors() { - List allPredecessors = new ArrayList(); - - for (DagConnection dagConnection : getIncomingConnections()) { - allPredecessors.add(dagConnection.getSource()); - } - - for (DagConnection conn : getBroadcastConnections()) { - allPredecessors.add(conn.getSource()); - } - - return allPredecessors; - } - - /** - * Gets the ID of this node. If the id has not yet been set, this method returns -1; - * - * @return This node's id, or -1, if not yet set. - */ - public int getId() { - return this.id; - } - - /** - * Sets the ID of this node. - * - * @param id The id for this node. - */ - public void initId(int id) { - if (id <= 0) { - throw new IllegalArgumentException(); - } - - if (this.id == -1) { - this.id = id; - } else { - throw new IllegalStateException("Id has already been initialized."); - } - } - - /** - * Adds the broadcast connection identified by the given {@code name} to this node. - * - * @param broadcastConnection The connection to add. - */ - public void addBroadcastConnection(String name, DagConnection broadcastConnection) { - this.broadcastConnectionNames.add(name); - this.broadcastConnections.add(broadcastConnection); - } - - /** Return the list of names associated with broadcast inputs for this node. */ - public List getBroadcastConnectionNames() { - return this.broadcastConnectionNames; - } - - /** Return the list of inputs associated with broadcast variables for this node. */ - public List getBroadcastConnections() { - return this.broadcastConnections; - } - - /** - * Adds a new outgoing connection to this node. - * - * @param connection The connection to add. - */ - public void addOutgoingConnection(DagConnection connection) { - if (this.outgoingConnections == null) { - this.outgoingConnections = new ArrayList(); - } else { - if (this.outgoingConnections.size() == 64) { - throw new CompilerException( - "Cannot currently handle nodes with more than 64 outputs."); - } - } - - this.outgoingConnections.add(connection); - } - - /** - * The list of outgoing connections from this node to succeeding tasks. - * - * @return The list of outgoing connections. - */ - public List getOutgoingConnections() { - return this.outgoingConnections; - } - - /** - * Gets the operator represented by this optimizer node. - * - * @return This node's operator. - */ - public Operator getOperator() { - return this.operator; - } - - /** - * Gets the parallelism for the operator represented by this optimizer node. The parallelism - * denotes how many parallel instances of the operator on will be spawned during the execution. - * If this value is {@link ExecutionConfig#PARALLELISM_DEFAULT} then the system will take the - * default number of parallel instances. - * - * @return The parallelism of the operator. - */ - public int getParallelism() { - return this.parallelism; - } - - /** - * Sets the parallelism for this optimizer node. The parallelism denotes how many parallel - * instances of the operator will be spawned during the execution. - * - * @param parallelism The parallelism to set. If this value is {@link - * ExecutionConfig#PARALLELISM_DEFAULT} then the system will take the default number of - * parallel instances. - * @throws IllegalArgumentException If the parallelism is smaller than one. - */ - public void setParallelism(int parallelism) { - if (parallelism < 1 && parallelism != ExecutionConfig.PARALLELISM_DEFAULT) { - throw new IllegalArgumentException("Parallelism of " + parallelism + " is invalid."); - } - this.parallelism = parallelism; - } - - /** - * Gets the amount of memory that all subtasks of this task have jointly available. - * - * @return The total amount of memory across all subtasks. - */ - public long getMinimalMemoryAcrossAllSubTasks() { - return this.minimalMemoryPerSubTask == -1 - ? -1 - : this.minimalMemoryPerSubTask * this.parallelism; - } - - public boolean isOnDynamicPath() { - return this.onDynamicPath; - } - - public void identifyDynamicPath(int costWeight) { - boolean anyDynamic = false; - boolean allDynamic = true; - - for (DagConnection conn : getIncomingConnections()) { - boolean dynamicIn = conn.isOnDynamicPath(); - anyDynamic |= dynamicIn; - allDynamic &= dynamicIn; - } - - for (DagConnection conn : getBroadcastConnections()) { - boolean dynamicIn = conn.isOnDynamicPath(); - anyDynamic |= dynamicIn; - allDynamic &= dynamicIn; - } - - if (anyDynamic) { - this.onDynamicPath = true; - this.costWeight = costWeight; - if (!allDynamic) { - // this node joins static and dynamic path. - // mark the connections where the source is not dynamic as cached - for (DagConnection conn : getIncomingConnections()) { - if (!conn.getSource().isOnDynamicPath()) { - conn.setMaterializationMode(conn.getMaterializationMode().makeCached()); - } - } - - // broadcast variables are always cached, because they stay unchanged available in - // the - // runtime context of the functions - } - } - } - - public int getCostWeight() { - return this.costWeight; - } - - public int getMaxDepth() { - int maxDepth = 0; - for (DagConnection conn : getIncomingConnections()) { - maxDepth = Math.max(maxDepth, conn.getMaxDepth()); - } - for (DagConnection conn : getBroadcastConnections()) { - maxDepth = Math.max(maxDepth, conn.getMaxDepth()); - } - - return maxDepth; - } - - /** - * Gets the properties that are interesting for this node to produce. - * - * @return The interesting properties for this node, or null, if not yet computed. - */ - public InterestingProperties getInterestingProperties() { - return this.intProps; - } - - @Override - public long getEstimatedOutputSize() { - return this.estimatedOutputSize; - } - - @Override - public long getEstimatedNumRecords() { - return this.estimatedNumRecords; - } - - public void setEstimatedOutputSize(long estimatedOutputSize) { - this.estimatedOutputSize = estimatedOutputSize; - } - - public void setEstimatedNumRecords(long estimatedNumRecords) { - this.estimatedNumRecords = estimatedNumRecords; - } - - @Override - public float getEstimatedAvgWidthPerOutputRecord() { - if (this.estimatedOutputSize > 0 && this.estimatedNumRecords > 0) { - return ((float) this.estimatedOutputSize) / this.estimatedNumRecords; - } else { - return -1.0f; - } - } - - /** - * Checks whether this node has branching output. A node's output is branched, if it has more - * than one output connection. - * - * @return True, if the node's output branches. False otherwise. - */ - public boolean isBranching() { - return getOutgoingConnections() != null && getOutgoingConnections().size() > 1; - } - - public void markAllOutgoingConnectionsAsPipelineBreaking() { - if (this.outgoingConnections == null) { - throw new IllegalStateException( - "The outgoing connections have not yet been initialized."); - } - for (DagConnection conn : getOutgoingConnections()) { - conn.markBreaksPipeline(); - } - } - - // ------------------------------------------------------------------------ - // Miscellaneous - // ------------------------------------------------------------------------ - - /** - * Checks, if all outgoing connections have their interesting properties set from their target - * nodes. - * - * @return True, if on all outgoing connections, the interesting properties are set. False - * otherwise. - */ - public boolean haveAllOutputConnectionInterestingProperties() { - for (DagConnection conn : getOutgoingConnections()) { - if (conn.getInterestingProperties() == null) { - return false; - } - } - return true; - } - - /** - * Computes all the interesting properties that are relevant to this node. The interesting - * properties are a union of the interesting properties on each outgoing connection. However, if - * two interesting properties on the outgoing connections overlap, the interesting properties - * will occur only once in this set. For that, this method deduplicates and merges the - * interesting properties. This method returns copies of the original interesting properties - * objects and leaves the original objects, contained by the connections, unchanged. - */ - public void computeUnionOfInterestingPropertiesFromSuccessors() { - List conns = getOutgoingConnections(); - if (conns.size() == 0) { - // no incoming, we have none ourselves - this.intProps = new InterestingProperties(); - } else { - this.intProps = conns.get(0).getInterestingProperties().clone(); - for (int i = 1; i < conns.size(); i++) { - this.intProps.addInterestingProperties(conns.get(i).getInterestingProperties()); - } - } - this.intProps.dropTrivials(); - } - - public void clearInterestingProperties() { - this.intProps = null; - for (DagConnection conn : getIncomingConnections()) { - conn.clearInterestingProperties(); - } - for (DagConnection conn : getBroadcastConnections()) { - conn.clearInterestingProperties(); - } - } - - /** - * Causes this node to compute its output estimates (such as number of rows, size in bytes) - * based on the inputs and the compiler hints. The compiler hints are instantiated with - * conservative default values which are used if no other values are provided. Nodes may access - * the statistics to determine relevant information. - * - * @param statistics The statistics object which may be accessed to get statistical information. - * The parameter may be null, if no statistics are available. - */ - public void computeOutputEstimates(DataStatistics statistics) { - // sanity checking - for (DagConnection c : getIncomingConnections()) { - if (c.getSource() == null) { - throw new CompilerException( - "Bug: Estimate computation called before inputs have been set."); - } - } - - // let every operator do its computation - computeOperatorSpecificDefaultEstimates(statistics); - - if (this.estimatedOutputSize < 0) { - this.estimatedOutputSize = -1; - } - if (this.estimatedNumRecords < 0) { - this.estimatedNumRecords = -1; - } - - // overwrite default estimates with hints, if given - if (getOperator() == null || getOperator().getCompilerHints() == null) { - return; - } - - CompilerHints hints = getOperator().getCompilerHints(); - if (hints.getOutputSize() >= 0) { - this.estimatedOutputSize = hints.getOutputSize(); - } - - if (hints.getOutputCardinality() >= 0) { - this.estimatedNumRecords = hints.getOutputCardinality(); - } - - if (hints.getFilterFactor() >= 0.0f) { - if (this.estimatedNumRecords >= 0) { - this.estimatedNumRecords = - (long) (this.estimatedNumRecords * hints.getFilterFactor()); - - if (this.estimatedOutputSize >= 0) { - this.estimatedOutputSize = - (long) (this.estimatedOutputSize * hints.getFilterFactor()); - } - } else if (this instanceof SingleInputNode) { - OptimizerNode pred = ((SingleInputNode) this).getPredecessorNode(); - if (pred != null && pred.getEstimatedNumRecords() >= 0) { - this.estimatedNumRecords = - (long) (pred.getEstimatedNumRecords() * hints.getFilterFactor()); - } - } - } - - // use the width to infer the cardinality (given size) and vice versa - if (hints.getAvgOutputRecordSize() >= 1) { - // the estimated number of rows based on size - if (this.estimatedNumRecords == -1 && this.estimatedOutputSize >= 0) { - this.estimatedNumRecords = - (long) (this.estimatedOutputSize / hints.getAvgOutputRecordSize()); - } else if (this.estimatedOutputSize == -1 && this.estimatedNumRecords >= 0) { - this.estimatedOutputSize = - (long) (this.estimatedNumRecords * hints.getAvgOutputRecordSize()); - } - } - } - - protected abstract void computeOperatorSpecificDefaultEstimates(DataStatistics statistics); - - // ------------------------------------------------------------------------ - // Reading of stub annotations - // ------------------------------------------------------------------------ - - /** - * Reads all stub annotations, i.e. which fields remain constant, what cardinality bounds the - * functions have, which fields remain unique. - */ - protected void readStubAnnotations() { - readUniqueFieldsAnnotation(); - } - - protected void readUniqueFieldsAnnotation() { - if (this.operator.getCompilerHints() != null) { - Set
uniqueFieldSets = operator.getCompilerHints().getUniqueFields(); - if (uniqueFieldSets != null) { - if (this.uniqueFields == null) { - this.uniqueFields = new HashSet
(); - } - this.uniqueFields.addAll(uniqueFieldSets); - } - } - } - - // ------------------------------------------------------------------------ - // Access of stub annotations - // ------------------------------------------------------------------------ - - /** Gets the FieldSets which are unique in the output of the node. */ - public Set
getUniqueFields() { - return this.uniqueFields == null ? Collections.
emptySet() : this.uniqueFields; - } - - // -------------------------------------------------------------------------------------------- - // Pruning - // -------------------------------------------------------------------------------------------- - - protected void prunePlanAlternatives(List plans) { - if (plans.isEmpty()) { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Too restrictive plan hints."); - } - // shortcut for the simple case - if (plans.size() == 1) { - return; - } - - // we can only compare plan candidates that made equal choices - // at the branching points. for each choice at a branching point, - // we need to keep the cheapest (wrt. interesting properties). - // if we do not keep candidates for each branch choice, we might not - // find branch compatible candidates when joining the branches back. - - // for pruning, we are quasi AFTER the node, so in the presence of - // branches, we need form the per-branch-choice groups by the choice - // they made at the latest un-joined branching node. Note that this is - // different from the check for branch compatibility of candidates, as - // this happens on the input sub-plans and hence BEFORE the node (therefore - // it is relevant to find the latest (partially) joined branch point. - - if (this.openBranches == null || this.openBranches.isEmpty()) { - prunePlanAlternativesWithCommonBranching(plans); - } else { - // partition the candidates into groups that made the same sub-plan candidate - // choice at the latest unclosed branch point - - final OptimizerNode[] branchDeterminers = new OptimizerNode[this.openBranches.size()]; - - for (int i = 0; i < branchDeterminers.length; i++) { - branchDeterminers[i] = - this.openBranches.get(this.openBranches.size() - 1 - i).getBranchingNode(); - } - - // this sorter sorts by the candidate choice at the branch point - Comparator sorter = - new Comparator() { - - @Override - public int compare(PlanNode o1, PlanNode o2) { - for (OptimizerNode branchDeterminer : branchDeterminers) { - PlanNode n1 = o1.getCandidateAtBranchPoint(branchDeterminer); - PlanNode n2 = o2.getCandidateAtBranchPoint(branchDeterminer); - int hash1 = System.identityHashCode(n1); - int hash2 = System.identityHashCode(n2); - - if (hash1 != hash2) { - return hash1 - hash2; - } - } - return 0; - } - }; - Collections.sort(plans, sorter); - - List result = new ArrayList(); - List turn = new ArrayList(); - - final PlanNode[] determinerChoice = new PlanNode[branchDeterminers.length]; - - while (!plans.isEmpty()) { - // take one as the determiner - turn.clear(); - PlanNode determiner = plans.remove(plans.size() - 1); - turn.add(determiner); - - for (int i = 0; i < determinerChoice.length; i++) { - determinerChoice[i] = - determiner.getCandidateAtBranchPoint(branchDeterminers[i]); - } - - // go backwards through the plans and find all that are equal - boolean stillEqual = true; - for (int k = plans.size() - 1; k >= 0 && stillEqual; k--) { - PlanNode toCheck = plans.get(k); - - for (int i = 0; i < branchDeterminers.length; i++) { - PlanNode checkerChoice = - toCheck.getCandidateAtBranchPoint(branchDeterminers[i]); - - if (checkerChoice != determinerChoice[i]) { - // not the same anymore - stillEqual = false; - break; - } - } - - if (stillEqual) { - // the same - plans.remove(k); - turn.add(toCheck); - } - } - - // now that we have only plans with the same branch alternatives, prune! - if (turn.size() > 1) { - prunePlanAlternativesWithCommonBranching(turn); - } - result.addAll(turn); - } - - // after all turns are complete - plans.clear(); - plans.addAll(result); - } - } - - protected void prunePlanAlternativesWithCommonBranching(List plans) { - // for each interesting property, which plans are cheapest - final RequestedGlobalProperties[] gps = - this.intProps - .getGlobalProperties() - .toArray( - new RequestedGlobalProperties - [this.intProps.getGlobalProperties().size()]); - final RequestedLocalProperties[] lps = - this.intProps - .getLocalProperties() - .toArray( - new RequestedLocalProperties - [this.intProps.getLocalProperties().size()]); - - final PlanNode[][] toKeep = new PlanNode[gps.length][]; - final PlanNode[] cheapestForGlobal = new PlanNode[gps.length]; - - PlanNode cheapest = null; // the overall cheapest plan - - // go over all plans from the list - for (PlanNode candidate : plans) { - // check if that plan is the overall cheapest - if (cheapest == null - || (cheapest.getCumulativeCosts().compareTo(candidate.getCumulativeCosts()) - > 0)) { - cheapest = candidate; - } - - // find the interesting global properties that this plan matches - for (int i = 0; i < gps.length; i++) { - if (gps[i].isMetBy(candidate.getGlobalProperties())) { - // the candidate meets the global property requirements. That means - // it has a chance that its local properties are re-used (they would be - // destroyed if global properties need to be established) - - if (cheapestForGlobal[i] == null - || (cheapestForGlobal[i] - .getCumulativeCosts() - .compareTo(candidate.getCumulativeCosts()) - > 0)) { - cheapestForGlobal[i] = candidate; - } - - final PlanNode[] localMatches; - if (toKeep[i] == null) { - localMatches = new PlanNode[lps.length]; - toKeep[i] = localMatches; - } else { - localMatches = toKeep[i]; - } - - for (int k = 0; k < lps.length; k++) { - if (lps[k].isMetBy(candidate.getLocalProperties())) { - final PlanNode previous = localMatches[k]; - if (previous == null - || previous.getCumulativeCosts() - .compareTo(candidate.getCumulativeCosts()) - > 0) { - // this one is cheaper! - localMatches[k] = candidate; - } - } - } - } - } - } - - // all plans are set now - plans.clear(); - - // add the cheapest plan - if (cheapest != null) { - plans.add(cheapest); - cheapest.setPruningMarker(); // remember that the plan is in the set - } - - // add all others, which are optimal for some interesting properties - for (int i = 0; i < gps.length; i++) { - if (toKeep[i] != null) { - final PlanNode[] localMatches = toKeep[i]; - for (final PlanNode n : localMatches) { - if (n != null && !n.isPruneMarkerSet()) { - n.setPruningMarker(); - plans.add(n); - } - } - } - if (cheapestForGlobal[i] != null) { - final PlanNode n = cheapestForGlobal[i]; - if (!n.isPruneMarkerSet()) { - n.setPruningMarker(); - plans.add(n); - } - } - } - } - - // -------------------------------------------------------------------------------------------- - // Handling of branches - // -------------------------------------------------------------------------------------------- - - public boolean hasUnclosedBranches() { - return this.openBranches != null && !this.openBranches.isEmpty(); - } - - public Set getClosedBranchingNodes() { - return this.closedBranchingNodes; - } - - public List getOpenBranches() { - return this.openBranches; - } - - protected List getBranchesForParent(DagConnection toParent) { - if (this.outgoingConnections.size() == 1) { - // return our own stack of open branches, because nothing is added - if (this.openBranches == null || this.openBranches.isEmpty()) { - return Collections.emptyList(); - } else { - return new ArrayList(this.openBranches); - } - } else if (this.outgoingConnections.size() > 1) { - // we branch add a branch info to the stack - List branches = new ArrayList(4); - if (this.openBranches != null) { - branches.addAll(this.openBranches); - } - - // find out, which output number the connection to the parent - int num; - for (num = 0; num < this.outgoingConnections.size(); num++) { - if (this.outgoingConnections.get(num) == toParent) { - break; - } - } - if (num >= this.outgoingConnections.size()) { - throw new CompilerException( - "Error in compiler: " - + "Parent to get branch info for is not contained in the outgoing connections."); - } - - // create the description and add it - long bitvector = 0x1L << num; - branches.add(new UnclosedBranchDescriptor(this, bitvector)); - return branches; - } else { - throw new CompilerException( - "Error in compiler: Cannot get branch info for successor in a node with no successors."); - } - } - - protected void removeClosedBranches(List openList) { - if (openList == null - || openList.isEmpty() - || this.closedBranchingNodes == null - || this.closedBranchingNodes.isEmpty()) { - return; - } - - Iterator it = openList.iterator(); - while (it.hasNext()) { - if (this.closedBranchingNodes.contains(it.next().getBranchingNode())) { - // this branch was already closed --> remove it from the list - it.remove(); - } - } - } - - protected void addClosedBranches(Set alreadyClosed) { - if (alreadyClosed == null || alreadyClosed.isEmpty()) { - return; - } - - if (this.closedBranchingNodes == null) { - this.closedBranchingNodes = new HashSet(alreadyClosed); - } else { - this.closedBranchingNodes.addAll(alreadyClosed); - } - } - - protected void addClosedBranch(OptimizerNode alreadyClosed) { - if (this.closedBranchingNodes == null) { - this.closedBranchingNodes = new HashSet(); - } - - this.closedBranchingNodes.add(alreadyClosed); - } - - /** - * Checks whether to candidate plans for the sub-plan of this node are comparable. The two - * alternative plans are comparable, if - * - *

a) There is no branch in the sub-plan of this node b) Both candidates have the same - * candidate as the child at the last open branch. - * - * @param plan1 The root node of the first candidate plan. - * @param plan2 The root node of the second candidate plan. - * @return True if the nodes are branch compatible in the inputs. - */ - protected boolean areBranchCompatible(PlanNode plan1, PlanNode plan2) { - if (plan1 == null || plan2 == null) { - throw new NullPointerException(); - } - - // if there is no open branch, the children are always compatible. - // in most plans, that will be the dominant case - if (this.hereJoinedBranches == null || this.hereJoinedBranches.isEmpty()) { - return true; - } - - for (OptimizerNode joinedBrancher : hereJoinedBranches) { - final PlanNode branch1Cand = plan1.getCandidateAtBranchPoint(joinedBrancher); - final PlanNode branch2Cand = plan2.getCandidateAtBranchPoint(joinedBrancher); - - if (branch1Cand != null && branch2Cand != null && branch1Cand != branch2Cand) { - return false; - } - } - return true; - } - - /** - * The node IDs are assigned in graph-traversal order (pre-order), hence, each list is sorted by - * ID in ascending order and all consecutive lists start with IDs in ascending order. - * - * @param markJoinedBranchesAsPipelineBreaking True, if the - */ - protected final boolean mergeLists( - List child1open, - List child2open, - List result, - boolean markJoinedBranchesAsPipelineBreaking) { - - // remove branches which have already been closed - removeClosedBranches(child1open); - removeClosedBranches(child2open); - - result.clear(); - - // check how many open branches we have. the cases: - // 1) if both are null or empty, the result is null - // 2) if one side is null (or empty), the result is the other side. - // 3) both are set, then we need to merge. - if (child1open == null || child1open.isEmpty()) { - if (child2open != null && !child2open.isEmpty()) { - result.addAll(child2open); - } - return false; - } - - if (child2open == null || child2open.isEmpty()) { - result.addAll(child1open); - return false; - } - - int index1 = child1open.size() - 1; - int index2 = child2open.size() - 1; - - boolean didCloseABranch = false; - - // as both lists (child1open and child2open) are sorted in ascending ID order - // we can do a merge-join-like loop which preserved the order in the result list - // and eliminates duplicates - while (index1 >= 0 || index2 >= 0) { - int id1 = -1; - int id2 = index2 >= 0 ? child2open.get(index2).getBranchingNode().getId() : -1; - - while (index1 >= 0 && (id1 = child1open.get(index1).getBranchingNode().getId()) > id2) { - result.add(child1open.get(index1)); - index1--; - } - while (index2 >= 0 && (id2 = child2open.get(index2).getBranchingNode().getId()) > id1) { - result.add(child2open.get(index2)); - index2--; - } - - // match: they share a common branching child - if (id1 == id2) { - didCloseABranch = true; - - // if this is the latest common child, remember it - OptimizerNode currBanchingNode = child1open.get(index1).getBranchingNode(); - - long vector1 = child1open.get(index1).getJoinedPathsVector(); - long vector2 = child2open.get(index2).getJoinedPathsVector(); - - // check if this is the same descriptor, (meaning that it contains the same paths) - // if it is the same, add it only once, otherwise process the join of the paths - if (vector1 == vector2) { - result.add(child1open.get(index1)); - } else { - // we merge (re-join) a branch - - // mark the branch as a point where we break the pipeline - if (markJoinedBranchesAsPipelineBreaking) { - currBanchingNode.markAllOutgoingConnectionsAsPipelineBreaking(); - } - - if (this.hereJoinedBranches == null) { - this.hereJoinedBranches = new ArrayList(2); - } - this.hereJoinedBranches.add(currBanchingNode); - - // see, if this node closes the branch - long joinedInputs = vector1 | vector2; - - // this is 2^size - 1, which is all bits set at positions 0..size-1 - long allInputs = (0x1L << currBanchingNode.getOutgoingConnections().size()) - 1; - - if (joinedInputs == allInputs) { - // closed - we can remove it from the stack - addClosedBranch(currBanchingNode); - } else { - // not quite closed - result.add(new UnclosedBranchDescriptor(currBanchingNode, joinedInputs)); - } - } - - index1--; - index2--; - } - } - - // merged. now we need to reverse the list, because we added the elements in reverse order - Collections.reverse(result); - return didCloseABranch; - } - - @Override - public OptimizerNode getOptimizerNode() { - return this; - } - - @Override - public PlanNode getPlanNode() { - return null; - } - - @Override - public Iterable> getDumpableInputs() { - List> allInputs = - new ArrayList>(); - - allInputs.addAll(getIncomingConnections()); - allInputs.addAll(getBroadcastConnections()); - - return allInputs; - } - - @Override - public String toString() { - StringBuilder bld = new StringBuilder(); - - bld.append(getOperatorName()); - bld.append(" (").append(getOperator().getName()).append(") "); - - int i = 1; - for (DagConnection conn : getIncomingConnections()) { - String shipStrategyName = - conn.getShipStrategy() == null ? "null" : conn.getShipStrategy().name(); - bld.append('(').append(i++).append(":").append(shipStrategyName).append(')'); - } - - return bld.toString(); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Description of an unclosed branch. An unclosed branch is when the data flow branched (one - * operator's result is consumed by multiple targets), but these different branches (targets) - * have not been joined together. - */ - public static final class UnclosedBranchDescriptor { - - protected OptimizerNode branchingNode; - - protected long joinedPathsVector; - - /** - * Creates a new branching descriptor. - * - * @param branchingNode The node where the branch occurred (the node with multiple outputs). - * @param joinedPathsVector A bit vector describing which branches are tracked by this - * descriptor. The bit vector is one, where the branch is tracked, zero otherwise. - */ - protected UnclosedBranchDescriptor(OptimizerNode branchingNode, long joinedPathsVector) { - this.branchingNode = branchingNode; - this.joinedPathsVector = joinedPathsVector; - } - - public OptimizerNode getBranchingNode() { - return this.branchingNode; - } - - public long getJoinedPathsVector() { - return this.joinedPathsVector; - } - - @Override - public String toString() { - return "(" + this.branchingNode.getOperator() + ") [" + this.joinedPathsVector + "]"; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java deleted file mode 100644 index be8618b950e6e..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OuterJoinNode.java +++ /dev/null @@ -1,219 +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.optimizer.dag; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase; -import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase.OuterJoinType; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.AbstractJoinDescriptor; -import org.apache.flink.optimizer.operators.HashFullOuterJoinBuildFirstDescriptor; -import org.apache.flink.optimizer.operators.HashFullOuterJoinBuildSecondDescriptor; -import org.apache.flink.optimizer.operators.HashLeftOuterJoinBuildFirstDescriptor; -import org.apache.flink.optimizer.operators.HashLeftOuterJoinBuildSecondDescriptor; -import org.apache.flink.optimizer.operators.HashRightOuterJoinBuildFirstDescriptor; -import org.apache.flink.optimizer.operators.HashRightOuterJoinBuildSecondDescriptor; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; -import org.apache.flink.optimizer.operators.SortMergeFullOuterJoinDescriptor; -import org.apache.flink.optimizer.operators.SortMergeLeftOuterJoinDescriptor; -import org.apache.flink.optimizer.operators.SortMergeRightOuterJoinDescriptor; - -import java.util.ArrayList; -import java.util.List; - -public class OuterJoinNode extends TwoInputNode { - - private List dataProperties; - - /** - * Creates a new two input node for the optimizer plan, representing the given operator. - * - * @param operator The operator that the optimizer DAG node should represent. - */ - public OuterJoinNode(OuterJoinOperatorBase operator) { - super(operator); - - this.dataProperties = getDataProperties(); - } - - private List getDataProperties() { - OuterJoinOperatorBase operator = getOperator(); - - OuterJoinType type = operator.getOuterJoinType(); - - JoinHint joinHint = operator.getJoinHint(); - joinHint = joinHint == null ? JoinHint.OPTIMIZER_CHOOSES : joinHint; - - List list; - switch (type) { - case LEFT: - list = createLeftOuterJoinDescriptors(joinHint); - break; - case RIGHT: - list = createRightOuterJoinDescriptors(joinHint); - break; - case FULL: - list = createFullOuterJoinDescriptors(joinHint); - break; - default: - throw new CompilerException("Unknown outer join type: " + type); - } - - Partitioner customPartitioner = operator.getCustomPartitioner(); - if (customPartitioner != null) { - for (OperatorDescriptorDual desc : list) { - ((AbstractJoinDescriptor) desc).setCustomPartitioner(customPartitioner); - } - } - return list; - } - - private List createLeftOuterJoinDescriptors(JoinHint hint) { - - List list = new ArrayList<>(); - switch (hint) { - case OPTIMIZER_CHOOSES: - list.add(new SortMergeLeftOuterJoinDescriptor(this.keys1, this.keys2, true)); - list.add( - new HashLeftOuterJoinBuildSecondDescriptor( - this.keys1, this.keys2, true, true)); - break; - case REPARTITION_SORT_MERGE: - list.add(new SortMergeLeftOuterJoinDescriptor(this.keys1, this.keys2, false)); - break; - case REPARTITION_HASH_SECOND: - list.add( - new HashLeftOuterJoinBuildSecondDescriptor( - this.keys1, this.keys2, false, true)); - break; - case BROADCAST_HASH_SECOND: - list.add( - new HashLeftOuterJoinBuildSecondDescriptor( - this.keys1, this.keys2, true, false)); - break; - case REPARTITION_HASH_FIRST: - list.add( - new HashLeftOuterJoinBuildFirstDescriptor( - this.keys1, this.keys2, false, true)); - break; - case BROADCAST_HASH_FIRST: - default: - throw new CompilerException("Invalid join hint: " + hint + " for left outer join"); - } - return list; - } - - private List createRightOuterJoinDescriptors(JoinHint hint) { - - List list = new ArrayList<>(); - switch (hint) { - case OPTIMIZER_CHOOSES: - list.add(new SortMergeRightOuterJoinDescriptor(this.keys1, this.keys2, true)); - list.add( - new HashRightOuterJoinBuildFirstDescriptor( - this.keys1, this.keys2, true, true)); - break; - case REPARTITION_SORT_MERGE: - list.add(new SortMergeRightOuterJoinDescriptor(this.keys1, this.keys2, false)); - break; - case REPARTITION_HASH_FIRST: - list.add( - new HashRightOuterJoinBuildFirstDescriptor( - this.keys1, this.keys2, false, true)); - break; - case BROADCAST_HASH_FIRST: - list.add( - new HashRightOuterJoinBuildFirstDescriptor( - this.keys1, this.keys2, true, false)); - break; - case REPARTITION_HASH_SECOND: - list.add( - new HashRightOuterJoinBuildSecondDescriptor( - this.keys1, this.keys2, false, true)); - break; - case BROADCAST_HASH_SECOND: - default: - throw new CompilerException("Invalid join hint: " + hint + " for right outer join"); - } - return list; - } - - private List createFullOuterJoinDescriptors(JoinHint hint) { - - List list = new ArrayList<>(); - switch (hint) { - case OPTIMIZER_CHOOSES: - list.add(new SortMergeFullOuterJoinDescriptor(this.keys1, this.keys2)); - break; - case REPARTITION_SORT_MERGE: - list.add(new SortMergeFullOuterJoinDescriptor(this.keys1, this.keys2)); - break; - case REPARTITION_HASH_FIRST: - list.add(new HashFullOuterJoinBuildFirstDescriptor(this.keys1, this.keys2)); - break; - case REPARTITION_HASH_SECOND: - list.add(new HashFullOuterJoinBuildSecondDescriptor(this.keys1, this.keys2)); - break; - case BROADCAST_HASH_FIRST: - case BROADCAST_HASH_SECOND: - default: - throw new CompilerException("Invalid join hint: " + hint + " for full outer join"); - } - return list; - } - - @Override - public OuterJoinOperatorBase getOperator() { - return (OuterJoinOperatorBase) super.getOperator(); - } - - @Override - protected List getPossibleProperties() { - return dataProperties; - } - - @Override - public String getOperatorName() { - return "Outer Join"; - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - long card1 = getFirstPredecessorNode().getEstimatedNumRecords(); - long card2 = getSecondPredecessorNode().getEstimatedNumRecords(); - - if (card1 < 0 || card2 < 0) { - this.estimatedNumRecords = -1; - } else { - this.estimatedNumRecords = Math.max(card1, card2); - } - - if (this.estimatedNumRecords >= 0) { - float width1 = getFirstPredecessorNode().getEstimatedAvgWidthPerOutputRecord(); - float width2 = getSecondPredecessorNode().getEstimatedAvgWidthPerOutputRecord(); - float width = (width1 <= 0 || width2 <= 0) ? -1 : width1 + width2; - - if (width > 0) { - this.estimatedOutputSize = (long) (width * this.estimatedNumRecords); - } - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java deleted file mode 100644 index 6a05f3f54fc5d..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java +++ /dev/null @@ -1,170 +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.optimizer.dag; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.PartitionOperatorBase; -import org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Preconditions; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of a Partition operator node. */ -public class PartitionNode extends SingleInputNode { - - private final List possibleProperties; - - public PartitionNode(PartitionOperatorBase operator) { - super(operator); - - OperatorDescriptorSingle descr = - new PartitionDescriptor( - this.getOperator().getPartitionMethod(), - this.keys, - operator.getOrdering(), - operator.getCustomPartitioner(), - operator.getDistribution()); - this.possibleProperties = Collections.singletonList(descr); - } - - @Override - public PartitionOperatorBase getOperator() { - return (PartitionOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Partition"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // partitioning does not change the number of records - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords(); - this.estimatedOutputSize = getPredecessorNode().getEstimatedOutputSize(); - } - - @Override - public SemanticProperties getSemanticProperties() { - return new SingleInputSemanticProperties.AllFieldsForwardedProperties(); - } - - // -------------------------------------------------------------------------------------------- - - public static class PartitionDescriptor extends OperatorDescriptorSingle { - - private final PartitionMethod pMethod; - private final Partitioner customPartitioner; - private final DataDistribution distribution; - private final Ordering ordering; - - public PartitionDescriptor( - PartitionMethod pMethod, - FieldSet pKeys, - Ordering ordering, - Partitioner customPartitioner, - DataDistribution distribution) { - super(pKeys); - - Preconditions.checkArgument( - pMethod != PartitionMethod.RANGE - || pKeys.equals(new FieldSet(ordering.getFieldPositions())), - "Partition keys must match the given ordering."); - - this.pMethod = pMethod; - this.customPartitioner = customPartitioner; - this.distribution = distribution; - this.ordering = ordering; - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.UNARY_NO_OP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode(node, "Partition", in, DriverStrategy.UNARY_NO_OP); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties rgps = new RequestedGlobalProperties(); - - switch (this.pMethod) { - case HASH: - rgps.setHashPartitioned(this.keys); - break; - case REBALANCE: - rgps.setForceRebalancing(); - break; - case CUSTOM: - rgps.setCustomPartitioned(this.keys, this.customPartitioner); - break; - case RANGE: - rgps.setRangePartitioned(ordering, distribution); - break; - default: - throw new IllegalArgumentException("Invalid partition method"); - } - - return Collections.singletonList(rgps); - } - - @Override - protected List createPossibleLocalProperties() { - // partitioning does not require any local property. - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - // the partition node is a no-operation operation, such that all global properties are - // preserved. - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - // the partition node is a no-operation operation, such that all global properties are - // preserved. - return lProps; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PlanCacheCleaner.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PlanCacheCleaner.java deleted file mode 100644 index 6462a9f44f328..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PlanCacheCleaner.java +++ /dev/null @@ -1,39 +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.optimizer.dag; - -import org.apache.flink.util.Visitor; - -final class PlanCacheCleaner implements Visitor { - - static final PlanCacheCleaner INSTANCE = new PlanCacheCleaner(); - - @Override - public boolean preVisit(OptimizerNode visitable) { - if (visitable.cachedPlans != null && visitable.isOnDynamicPath()) { - visitable.cachedPlans = null; - return true; - } else { - return false; - } - } - - @Override - public void postVisit(OptimizerNode visitable) {} -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java deleted file mode 100644 index 25fce3e1c8787..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/ReduceNode.java +++ /dev/null @@ -1,120 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.AllReduceProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; -import org.apache.flink.optimizer.operators.ReduceProperties; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** The Optimizer representation of a Reduce operator. */ -public class ReduceNode extends SingleInputNode { - - private final List possibleProperties; - - private ReduceNode preReduceUtilityNode; - - public ReduceNode(ReduceOperatorBase operator) { - super(operator); - - if (this.keys == null) { - // case of a key-less reducer. force a parallelism of 1 - setParallelism(1); - } - - OperatorDescriptorSingle props; - - if (this.keys == null) { - props = new AllReduceProperties(); - } else { - DriverStrategy combinerStrategy; - switch (operator.getCombineHint()) { - case OPTIMIZER_CHOOSES: - combinerStrategy = DriverStrategy.SORTED_PARTIAL_REDUCE; - break; - case SORT: - combinerStrategy = DriverStrategy.SORTED_PARTIAL_REDUCE; - break; - case HASH: - combinerStrategy = DriverStrategy.HASHED_PARTIAL_REDUCE; - break; - case NONE: - combinerStrategy = DriverStrategy.NONE; - break; - default: - throw new RuntimeException("Unknown CombineHint"); - } - props = - new ReduceProperties( - this.keys, operator.getCustomPartitioner(), combinerStrategy); - } - - this.possibleProperties = Collections.singletonList(props); - } - - public ReduceNode(ReduceNode reducerToCopyForCombiner) { - super(reducerToCopyForCombiner); - - this.possibleProperties = Collections.emptyList(); - } - - // ------------------------------------------------------------------------ - - @Override - public ReduceOperatorBase getOperator() { - return (ReduceOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Reduce"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - // -------------------------------------------------------------------------------------------- - // Estimates - // -------------------------------------------------------------------------------------------- - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // no real estimates possible for a reducer. - } - - public ReduceNode getCombinerUtilityNode() { - if (this.preReduceUtilityNode == null) { - this.preReduceUtilityNode = new ReduceNode(this); - - // we conservatively assume the combiner returns the same data size as it consumes - this.preReduceUtilityNode.estimatedOutputSize = - getPredecessorNode().getEstimatedOutputSize(); - this.preReduceUtilityNode.estimatedNumRecords = - getPredecessorNode().getEstimatedNumRecords(); - } - return this.preReduceUtilityNode; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java deleted file mode 100644 index 3cf58463cee35..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java +++ /dev/null @@ -1,570 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.util.NoOpUnaryUdfOp; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -import org.apache.flink.shaded.guava32.com.google.common.collect.Sets; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** - * A node in the optimizer's program representation for an operation with a single input. - * - *

This class contains all the generic logic for handling branching flows, as well as to - * enumerate candidate execution plans. The subclasses for specific operators simply add logic for - * cost estimates and specify possible strategies for their execution. - */ -public abstract class SingleInputNode extends OptimizerNode { - - protected final FieldSet keys; // The set of key fields - - protected DagConnection inConn; // the input of the node - - // -------------------------------------------------------------------------------------------- - - /** - * Creates a new node with a single input for the optimizer plan. - * - * @param programOperator The PACT that the node represents. - */ - protected SingleInputNode(SingleInputOperator programOperator) { - super(programOperator); - - int[] k = programOperator.getKeyColumns(0); - this.keys = k == null || k.length == 0 ? null : new FieldSet(k); - } - - protected SingleInputNode(FieldSet keys) { - super(NoOpUnaryUdfOp.INSTANCE); - this.keys = keys; - } - - protected SingleInputNode() { - super(NoOpUnaryUdfOp.INSTANCE); - this.keys = null; - } - - protected SingleInputNode(SingleInputNode toCopy) { - super(toCopy); - - this.keys = toCopy.keys; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public SingleInputOperator getOperator() { - return (SingleInputOperator) super.getOperator(); - } - - /** - * Gets the input of this operator. - * - * @return The input. - */ - public DagConnection getIncomingConnection() { - return this.inConn; - } - - /** - * Sets the connection through which this node receives its input. - * - * @param inConn The input connection to set. - */ - public void setIncomingConnection(DagConnection inConn) { - this.inConn = inConn; - } - - /** - * Gets the predecessor of this node. - * - * @return The predecessor of this node. - */ - public OptimizerNode getPredecessorNode() { - if (this.inConn != null) { - return this.inConn.getSource(); - } else { - return null; - } - } - - @Override - public List getIncomingConnections() { - return Collections.singletonList(this.inConn); - } - - @Override - public SemanticProperties getSemanticProperties() { - return getOperator().getSemanticProperties(); - } - - protected SemanticProperties getSemanticPropertiesForLocalPropertyFiltering() { - return this.getSemanticProperties(); - } - - protected SemanticProperties getSemanticPropertiesForGlobalPropertyFiltering() { - return this.getSemanticProperties(); - } - - @Override - public void setInput( - Map, OptimizerNode> contractToNode, ExecutionMode defaultExchangeMode) - throws CompilerException { - // see if an internal hint dictates the strategy to use - final Configuration conf = getOperator().getParameters(); - final String shipStrategy = conf.getString(Optimizer.HINT_SHIP_STRATEGY, null); - final ShipStrategyType preSet; - - if (shipStrategy != null) { - if (shipStrategy.equalsIgnoreCase(Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH)) { - preSet = ShipStrategyType.PARTITION_HASH; - } else if (shipStrategy.equalsIgnoreCase( - Optimizer.HINT_SHIP_STRATEGY_REPARTITION_RANGE)) { - preSet = ShipStrategyType.PARTITION_RANGE; - } else if (shipStrategy.equalsIgnoreCase(Optimizer.HINT_SHIP_STRATEGY_FORWARD)) { - preSet = ShipStrategyType.FORWARD; - } else if (shipStrategy.equalsIgnoreCase(Optimizer.HINT_SHIP_STRATEGY_REPARTITION)) { - preSet = ShipStrategyType.PARTITION_RANDOM; - } else { - throw new CompilerException("Unrecognized ship strategy hint: " + shipStrategy); - } - } else { - preSet = null; - } - - // get the predecessor node - Operator children = ((SingleInputOperator) getOperator()).getInput(); - - OptimizerNode pred; - DagConnection conn; - if (children == null) { - throw new CompilerException( - "Error: Node for '" + getOperator().getName() + "' has no input."); - } else { - pred = contractToNode.get(children); - conn = new DagConnection(pred, this, defaultExchangeMode); - if (preSet != null) { - conn.setShipStrategy(preSet); - } - } - - // create the connection and add it - setIncomingConnection(conn); - pred.addOutgoingConnection(conn); - } - - // -------------------------------------------------------------------------------------------- - // Properties and Optimization - // -------------------------------------------------------------------------------------------- - - protected abstract List getPossibleProperties(); - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - // get what we inherit and what is preserved by our user code - final InterestingProperties props = - getInterestingProperties().filterByCodeAnnotations(this, 0); - - // add all properties relevant to this node - for (OperatorDescriptorSingle dps : getPossibleProperties()) { - for (RequestedGlobalProperties gp : dps.getPossibleGlobalProperties()) { - - if (gp.getPartitioning().isPartitionedOnKey()) { - // make sure that among the same partitioning types, we do not push anything - // down that has fewer key fields - - for (RequestedGlobalProperties contained : props.getGlobalProperties()) { - if (contained.getPartitioning() == gp.getPartitioning() - && gp.getPartitionedFields() - .isValidSubset(contained.getPartitionedFields())) { - props.getGlobalProperties().remove(contained); - break; - } - } - } - - props.addGlobalProperties(gp); - } - - for (RequestedLocalProperties lp : dps.getPossibleLocalProperties()) { - props.addLocalProperties(lp); - } - } - this.inConn.setInterestingProperties(props); - - for (DagConnection conn : getBroadcastConnections()) { - conn.setInterestingProperties(new InterestingProperties()); - } - } - - @Override - public List getAlternativePlans(CostEstimator estimator) { - // check if we have a cached version - if (this.cachedPlans != null) { - return this.cachedPlans; - } - - boolean childrenSkippedDueToReplicatedInput = false; - - // calculate alternative sub-plans for predecessor - final List subPlans = - getPredecessorNode().getAlternativePlans(estimator); - final Set intGlobal = - this.inConn.getInterestingProperties().getGlobalProperties(); - - // calculate alternative sub-plans for broadcast inputs - final List> broadcastPlanChannels = - new ArrayList>(); - List broadcastConnections = getBroadcastConnections(); - List broadcastConnectionNames = getBroadcastConnectionNames(); - - for (int i = 0; i < broadcastConnections.size(); i++) { - DagConnection broadcastConnection = broadcastConnections.get(i); - String broadcastConnectionName = broadcastConnectionNames.get(i); - List broadcastPlanCandidates = - broadcastConnection.getSource().getAlternativePlans(estimator); - - // wrap the plan candidates in named channels - HashSet broadcastChannels = - new HashSet(broadcastPlanCandidates.size()); - for (PlanNode plan : broadcastPlanCandidates) { - NamedChannel c = new NamedChannel(broadcastConnectionName, plan); - DataExchangeMode exMode = - DataExchangeMode.select( - broadcastConnection.getDataExchangeMode(), - ShipStrategyType.BROADCAST, - broadcastConnection.isBreakingPipeline()); - c.setShipStrategy(ShipStrategyType.BROADCAST, exMode); - broadcastChannels.add(c); - } - broadcastPlanChannels.add(broadcastChannels); - } - - final RequestedGlobalProperties[] allValidGlobals; - { - Set pairs = new HashSet(); - for (OperatorDescriptorSingle ods : getPossibleProperties()) { - pairs.addAll(ods.getPossibleGlobalProperties()); - } - allValidGlobals = pairs.toArray(new RequestedGlobalProperties[pairs.size()]); - } - final ArrayList outputPlans = new ArrayList(); - - final ExecutionMode executionMode = this.inConn.getDataExchangeMode(); - - final int parallelism = getParallelism(); - final int inParallelism = getPredecessorNode().getParallelism(); - - final boolean parallelismChange = inParallelism != parallelism; - - final boolean breaksPipeline = this.inConn.isBreakingPipeline(); - - // create all candidates - for (PlanNode child : subPlans) { - - if (child.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if the parallelism is not - // changed - if (parallelismChange) { - // can not continue with this child - childrenSkippedDueToReplicatedInput = true; - continue; - } else { - this.inConn.setShipStrategy(ShipStrategyType.FORWARD); - } - } - - if (this.inConn.getShipStrategy() == null) { - // pick the strategy ourselves - for (RequestedGlobalProperties igps : intGlobal) { - final Channel c = new Channel(child, this.inConn.getMaterializationMode()); - igps.parameterizeChannel(c, parallelismChange, executionMode, breaksPipeline); - - // if the parallelism changed, make sure that we cancel out properties, unless - // the - // ship strategy preserves/establishes them even under changing parallelisms - if (parallelismChange && !c.getShipStrategy().isNetworkStrategy()) { - c.getGlobalProperties().reset(); - } - - // check whether we meet any of the accepted properties - // we may remove this check, when we do a check to not inherit - // requested global properties that are incompatible with all possible - // requested properties - for (RequestedGlobalProperties rgps : allValidGlobals) { - if (rgps.isMetBy(c.getGlobalProperties())) { - c.setRequiredGlobalProps(rgps); - addLocalCandidates( - c, broadcastPlanChannels, igps, outputPlans, estimator); - break; - } - } - } - } else { - // hint fixed the strategy - final Channel c = new Channel(child, this.inConn.getMaterializationMode()); - final ShipStrategyType shipStrategy = this.inConn.getShipStrategy(); - final DataExchangeMode exMode = - DataExchangeMode.select(executionMode, shipStrategy, breaksPipeline); - - if (this.keys != null) { - c.setShipStrategy(shipStrategy, this.keys.toFieldList(), exMode); - } else { - c.setShipStrategy(shipStrategy, exMode); - } - - if (parallelismChange) { - c.adjustGlobalPropertiesForFullParallelismChange(); - } - - // check whether we meet any of the accepted properties - for (RequestedGlobalProperties rgps : allValidGlobals) { - if (rgps.isMetBy(c.getGlobalProperties())) { - addLocalCandidates(c, broadcastPlanChannels, rgps, outputPlans, estimator); - break; - } - } - } - } - - if (outputPlans.isEmpty()) { - if (childrenSkippedDueToReplicatedInput) { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Invalid use of replicated input."); - } else { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Too restrictive plan hints."); - } - } - - // cost and prune the plans - for (PlanNode node : outputPlans) { - estimator.costOperator(node); - } - prunePlanAlternatives(outputPlans); - outputPlans.trimToSize(); - - this.cachedPlans = outputPlans; - return outputPlans; - } - - protected void addLocalCandidates( - Channel template, - List> broadcastPlanChannels, - RequestedGlobalProperties rgps, - List target, - CostEstimator estimator) { - for (RequestedLocalProperties ilp : - this.inConn.getInterestingProperties().getLocalProperties()) { - final Channel in = template.clone(); - ilp.parameterizeChannel(in); - - // instantiate a candidate, if the instantiated local properties meet one possible local - // property set - outer: - for (OperatorDescriptorSingle dps : getPossibleProperties()) { - for (RequestedLocalProperties ilps : dps.getPossibleLocalProperties()) { - if (ilps.isMetBy(in.getLocalProperties())) { - in.setRequiredLocalProps(ilps); - instantiateCandidate( - dps, in, broadcastPlanChannels, target, estimator, rgps, ilp); - break outer; - } - } - } - } - } - - protected void instantiateCandidate( - OperatorDescriptorSingle dps, - Channel in, - List> broadcastPlanChannels, - List target, - CostEstimator estimator, - RequestedGlobalProperties globPropsReq, - RequestedLocalProperties locPropsReq) { - final PlanNode inputSource = in.getSource(); - - for (List broadcastChannelsCombination : - Sets.cartesianProduct(broadcastPlanChannels)) { - - boolean validCombination = true; - boolean requiresPipelinebreaker = false; - - // check whether the broadcast inputs use the same plan candidate at the branching point - for (int i = 0; i < broadcastChannelsCombination.size(); i++) { - NamedChannel nc = broadcastChannelsCombination.get(i); - PlanNode bcSource = nc.getSource(); - - // check branch compatibility against input - if (!areBranchCompatible(bcSource, inputSource)) { - validCombination = false; - break; - } - - // check branch compatibility against all other broadcast variables - for (int k = 0; k < i; k++) { - PlanNode otherBcSource = broadcastChannelsCombination.get(k).getSource(); - - if (!areBranchCompatible(bcSource, otherBcSource)) { - validCombination = false; - break; - } - } - - // check if there is a common predecessor and whether there is a dam on the way to - // all common predecessors - if (in.isOnDynamicPath() && this.hereJoinedBranches != null) { - for (OptimizerNode brancher : this.hereJoinedBranches) { - PlanNode candAtBrancher = - in.getSource().getCandidateAtBranchPoint(brancher); - - if (candAtBrancher == null) { - // closed branch between two broadcast variables - continue; - } - - SourceAndDamReport res = in.getSource().hasDamOnPathDownTo(candAtBrancher); - if (res == NOT_FOUND) { - throw new CompilerException( - "Bug: Tracing dams for deadlock detection is broken."); - } else if (res == FOUND_SOURCE) { - requiresPipelinebreaker = true; - break; - } else if (res == FOUND_SOURCE_AND_DAM) { - // good - } else { - throw new CompilerException(); - } - } - } - } - - if (!validCombination) { - continue; - } - - if (requiresPipelinebreaker) { - in.setTempMode(in.getTempMode().makePipelineBreaker()); - } - - final SingleInputPlanNode node = dps.instantiate(in, this); - node.setBroadcastInputs(broadcastChannelsCombination); - - // compute how the strategy affects the properties - GlobalProperties gProps = in.getGlobalProperties().clone(); - LocalProperties lProps = in.getLocalProperties().clone(); - gProps = dps.computeGlobalProperties(gProps); - lProps = dps.computeLocalProperties(lProps); - - // filter by the user code field copies - gProps = - gProps.filterBySemanticProperties( - getSemanticPropertiesForGlobalPropertyFiltering(), 0); - lProps = - lProps.filterBySemanticProperties( - getSemanticPropertiesForLocalPropertyFiltering(), 0); - - // apply - node.initProperties(gProps, lProps); - node.updatePropertiesWithUniqueSets(getUniqueFields()); - target.add(node); - } - } - - // -------------------------------------------------------------------------------------------- - // Branch Handling - // -------------------------------------------------------------------------------------------- - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - addClosedBranches(getPredecessorNode().closedBranchingNodes); - List fromInput = - getPredecessorNode().getBranchesForParent(this.inConn); - - // handle the data flow branching for the broadcast inputs - List result = - computeUnclosedBranchStackForBroadcastInputs(fromInput); - - this.openBranches = - (result == null || result.isEmpty()) - ? Collections.emptyList() - : result; - } - - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - if (getPredecessorNode() != null) { - getPredecessorNode().accept(visitor); - } else { - throw new CompilerException(); - } - for (DagConnection connection : getBroadcastConnections()) { - connection.getSource().accept(visitor); - } - visitor.postVisit(this); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java deleted file mode 100644 index 0275d01469fe3..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SinkJoiner.java +++ /dev/null @@ -1,116 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.typeinfo.NothingTypeInfo; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; -import org.apache.flink.optimizer.operators.UtilSinkJoinOpDescriptor; -import org.apache.flink.optimizer.util.NoOpBinaryUdfOp; -import org.apache.flink.types.Nothing; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** - * This class represents a utility node that is not part of the actual plan. It is used for plans - * with multiple data sinks to transform it into a plan with a single root node. That way, the code - * that makes sure no costs are double-counted and that candidate selection works correctly with - * nodes that have multiple outputs is transparently reused. - */ -public class SinkJoiner extends TwoInputNode { - - public SinkJoiner(OptimizerNode input1, OptimizerNode input2) { - super(new NoOpBinaryUdfOp(new NothingTypeInfo())); - - DagConnection conn1 = new DagConnection(input1, this, null, ExecutionMode.PIPELINED); - DagConnection conn2 = new DagConnection(input2, this, null, ExecutionMode.PIPELINED); - - this.input1 = conn1; - this.input2 = conn2; - - setParallelism(1); - } - - @Override - public String getOperatorName() { - return "Internal Utility Node"; - } - - @Override - public List getOutgoingConnections() { - return Collections.emptyList(); - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - addClosedBranches(getFirstPredecessorNode().closedBranchingNodes); - addClosedBranches(getSecondPredecessorNode().closedBranchingNodes); - - List pred1branches = getFirstPredecessorNode().openBranches; - List pred2branches = getSecondPredecessorNode().openBranches; - - // if the predecessors do not have branches, then we have multiple sinks that do not - // originate from - // a common data flow. - if (pred1branches == null || pred1branches.isEmpty()) { - - this.openBranches = - (pred2branches == null || pred2branches.isEmpty()) - ? Collections.emptyList() - : // both empty - disconnected flow - pred2branches; - } else if (pred2branches == null || pred2branches.isEmpty()) { - this.openBranches = pred1branches; - } else { - // copy the lists and merge - List result1 = - new ArrayList(pred1branches); - List result2 = - new ArrayList(pred2branches); - - ArrayList result = new ArrayList(); - mergeLists(result1, result2, result, false); - - this.openBranches = - result.isEmpty() ? Collections.emptyList() : result; - } - } - - @Override - protected List getPossibleProperties() { - return Collections.singletonList(new UtilSinkJoinOpDescriptor()); - } - - @Override - public void computeOutputEstimates(DataStatistics statistics) { - // nothing to be done here - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // no estimates needed at this point - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SolutionSetNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SolutionSetNode.java deleted file mode 100644 index 107ae7279a5e3..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SolutionSetNode.java +++ /dev/null @@ -1,107 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.base.DeltaIterationBase.SolutionSetPlaceHolder; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SolutionSetPlanNode; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of the solution set of a workset iteration. */ -public class SolutionSetNode extends AbstractPartialSolutionNode { - - private final WorksetIterationNode iterationNode; - - public SolutionSetNode(SolutionSetPlaceHolder psph, WorksetIterationNode iterationNode) { - super(psph); - this.iterationNode = iterationNode; - } - - // -------------------------------------------------------------------------------------------- - - public void setCandidateProperties( - GlobalProperties gProps, LocalProperties lProps, Channel initialInput) { - this.cachedPlans = - Collections.singletonList( - new SolutionSetPlanNode( - this, - "SolutionSet (" + this.getOperator().getName() + ")", - gProps, - lProps, - initialInput)); - } - - public SolutionSetPlanNode getCurrentSolutionSetPlanNode() { - if (this.cachedPlans != null) { - return (SolutionSetPlanNode) this.cachedPlans.get(0); - } else { - throw new IllegalStateException(); - } - } - - public WorksetIterationNode getIterationNode() { - return this.iterationNode; - } - - @Override - public void computeOutputEstimates(DataStatistics statistics) { - copyEstimates(this.iterationNode.getInitialSolutionSetPredecessorNode()); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the contract object for this data source node. - * - * @return The contract. - */ - @Override - public SolutionSetPlaceHolder getOperator() { - return (SolutionSetPlaceHolder) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Solution Set"; - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - DagConnection solutionSetInput = this.iterationNode.getFirstIncomingConnection(); - OptimizerNode solutionSetSource = solutionSetInput.getSource(); - - addClosedBranches(solutionSetSource.closedBranchingNodes); - List fromInput = - solutionSetSource.getBranchesForParent(solutionSetInput); - this.openBranches = - (fromInput == null || fromInput.isEmpty()) - ? Collections.emptyList() - : fromInput; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SortPartitionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SortPartitionNode.java deleted file mode 100644 index 0e6f86b119a6b..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SortPartitionNode.java +++ /dev/null @@ -1,127 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.SortPartitionOperatorBase; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** The optimizer's internal representation of a SortPartition operator node. */ -public class SortPartitionNode extends SingleInputNode { - - private final List possibleProperties; - - public SortPartitionNode(SortPartitionOperatorBase operator) { - super(operator); - - OperatorDescriptorSingle descr = - new SortPartitionDescriptor(operator.getPartitionOrdering()); - this.possibleProperties = Collections.singletonList(descr); - } - - @Override - public SortPartitionOperatorBase getOperator() { - return (SortPartitionOperatorBase) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Sort-Partition"; - } - - @Override - protected List getPossibleProperties() { - return this.possibleProperties; - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // sorting does not change the number of records - this.estimatedNumRecords = getPredecessorNode().getEstimatedNumRecords(); - this.estimatedOutputSize = getPredecessorNode().getEstimatedOutputSize(); - } - - @Override - public SemanticProperties getSemanticProperties() { - return new SingleInputSemanticProperties.AllFieldsForwardedProperties(); - } - - // -------------------------------------------------------------------------------------------- - - public static class SortPartitionDescriptor extends OperatorDescriptorSingle { - - private Ordering partitionOrder; - - public SortPartitionDescriptor(Ordering partitionOrder) { - this.partitionOrder = partitionOrder; - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.UNARY_NO_OP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode(node, "Sort-Partition", in, DriverStrategy.UNARY_NO_OP); - } - - @Override - protected List createPossibleGlobalProperties() { - // sort partition does not require any global property - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - // set partition order as required local property - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setOrdering(this.partitionOrder); - - return Collections.singletonList(rlp); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - // sort partition is a no-operation operation, such that all global properties are - // preserved. - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - // sort partition is a no-operation operation, such that all local properties are - // preserved. - return lProps; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TempMode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TempMode.java deleted file mode 100644 index e609d8cd7e66a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TempMode.java +++ /dev/null @@ -1,80 +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.optimizer.dag; - -/** - * Enumeration to indicate the mode of temporarily materializing the data that flows across a - * connection. Introducing such an artificial dam is sometimes necessary to avoid that a certain - * data flows deadlock themselves, or as a cache to replay an intermediate result. - */ -public enum TempMode { - NONE(false, false), - PIPELINE_BREAKER(false, true), - CACHED(true, false), - CACHING_PIPELINE_BREAKER(true, true); - - // -------------------------------------------------------------------------------------------- - - private final boolean cached; - - private final boolean breaksPipeline; - - private TempMode(boolean cached, boolean breaksPipeline) { - this.cached = cached; - this.breaksPipeline = breaksPipeline; - } - - public boolean isCached() { - return cached; - } - - public boolean breaksPipeline() { - return breaksPipeline; - } - - public TempMode makePipelineBreaker() { - if (this == NONE) { - return PIPELINE_BREAKER; - } else if (this == CACHED) { - return CACHING_PIPELINE_BREAKER; - } else { - return this; - } - } - - public TempMode makeCached() { - if (this == NONE) { - return CACHED; - } else if (this == PIPELINE_BREAKER) { - return CACHING_PIPELINE_BREAKER; - } else { - return this; - } - } - - public TempMode makeNonCached() { - if (this == CACHED) { - return NONE; - } else if (this == CACHING_PIPELINE_BREAKER) { - return PIPELINE_BREAKER; - } else { - return this; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java deleted file mode 100644 index 6cf187a88989b..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java +++ /dev/null @@ -1,860 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.DualInputOperator; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual.GlobalPropertiesPair; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual.LocalPropertiesPair; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DamBehavior; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -import org.apache.flink.shaded.guava32.com.google.common.collect.Sets; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** - * A node in the optimizer plan that represents an operator with a two different inputs, such as - * Join, Cross, CoGroup, or Union. The two inputs are not substitutable in their sides. - */ -public abstract class TwoInputNode extends OptimizerNode { - - protected final FieldList keys1; // The set of key fields for the first input. may be null. - - protected final FieldList keys2; // The set of key fields for the second input. may be null. - - protected DagConnection input1; // The first input edge - - protected DagConnection input2; // The second input edge - - private List cachedDescriptors; - - // -------------------------------------------------------------------------------------------- - - /** - * Creates a new two input node for the optimizer plan, representing the given operator. - * - * @param operator The operator that the optimizer DAG node should represent. - */ - public TwoInputNode(DualInputOperator operator) { - super(operator); - - int[] k1 = operator.getKeyColumns(0); - int[] k2 = operator.getKeyColumns(1); - - this.keys1 = k1 == null || k1.length == 0 ? null : new FieldList(k1); - this.keys2 = k2 == null || k2.length == 0 ? null : new FieldList(k2); - - if (this.keys1 != null) { - if (this.keys2 != null) { - if (this.keys1.size() != this.keys2.size()) { - throw new CompilerException("Unequal number of key fields on the two inputs."); - } - } else { - throw new CompilerException("Keys are set on first input, but not on second."); - } - } else if (this.keys2 != null) { - throw new CompilerException("Keys are set on second input, but not on first."); - } - } - - // ------------------------------------------------------------------------ - - @Override - public DualInputOperator getOperator() { - return (DualInputOperator) super.getOperator(); - } - - /** - * Gets the DagConnection through which this node receives its first input. - * - * @return The first input connection. - */ - public DagConnection getFirstIncomingConnection() { - return this.input1; - } - - /** - * Gets the DagConnection through which this node receives its second input. - * - * @return The second input connection. - */ - public DagConnection getSecondIncomingConnection() { - return this.input2; - } - - public OptimizerNode getFirstPredecessorNode() { - if (this.input1 != null) { - return this.input1.getSource(); - } else { - return null; - } - } - - public OptimizerNode getSecondPredecessorNode() { - if (this.input2 != null) { - return this.input2.getSource(); - } else { - return null; - } - } - - @Override - public List getIncomingConnections() { - ArrayList inputs = new ArrayList(2); - inputs.add(input1); - inputs.add(input2); - return inputs; - } - - @Override - public void setInput( - Map, OptimizerNode> contractToNode, ExecutionMode defaultExecutionMode) { - // see if there is a hint that dictates which shipping strategy to use for BOTH inputs - final Configuration conf = getOperator().getParameters(); - ShipStrategyType preSet1 = null; - ShipStrategyType preSet2 = null; - - String shipStrategy = conf.getString(Optimizer.HINT_SHIP_STRATEGY, null); - if (shipStrategy != null) { - if (Optimizer.HINT_SHIP_STRATEGY_FORWARD.equals(shipStrategy)) { - preSet1 = preSet2 = ShipStrategyType.FORWARD; - } else if (Optimizer.HINT_SHIP_STRATEGY_BROADCAST.equals(shipStrategy)) { - preSet1 = preSet2 = ShipStrategyType.BROADCAST; - } else if (Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH.equals(shipStrategy)) { - preSet1 = preSet2 = ShipStrategyType.PARTITION_HASH; - } else if (Optimizer.HINT_SHIP_STRATEGY_REPARTITION_RANGE.equals(shipStrategy)) { - preSet1 = preSet2 = ShipStrategyType.PARTITION_RANGE; - } else if (shipStrategy.equalsIgnoreCase(Optimizer.HINT_SHIP_STRATEGY_REPARTITION)) { - preSet1 = preSet2 = ShipStrategyType.PARTITION_RANDOM; - } else { - throw new CompilerException("Unknown hint for shipping strategy: " + shipStrategy); - } - } - - // see if there is a hint that dictates which shipping strategy to use for the FIRST input - shipStrategy = conf.getString(Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, null); - if (shipStrategy != null) { - if (Optimizer.HINT_SHIP_STRATEGY_FORWARD.equals(shipStrategy)) { - preSet1 = ShipStrategyType.FORWARD; - } else if (Optimizer.HINT_SHIP_STRATEGY_BROADCAST.equals(shipStrategy)) { - preSet1 = ShipStrategyType.BROADCAST; - } else if (Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH.equals(shipStrategy)) { - preSet1 = ShipStrategyType.PARTITION_HASH; - } else if (Optimizer.HINT_SHIP_STRATEGY_REPARTITION_RANGE.equals(shipStrategy)) { - preSet1 = ShipStrategyType.PARTITION_RANGE; - } else if (shipStrategy.equalsIgnoreCase(Optimizer.HINT_SHIP_STRATEGY_REPARTITION)) { - preSet1 = ShipStrategyType.PARTITION_RANDOM; - } else { - throw new CompilerException( - "Unknown hint for shipping strategy of input one: " + shipStrategy); - } - } - - // see if there is a hint that dictates which shipping strategy to use for the SECOND input - shipStrategy = conf.getString(Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, null); - if (shipStrategy != null) { - if (Optimizer.HINT_SHIP_STRATEGY_FORWARD.equals(shipStrategy)) { - preSet2 = ShipStrategyType.FORWARD; - } else if (Optimizer.HINT_SHIP_STRATEGY_BROADCAST.equals(shipStrategy)) { - preSet2 = ShipStrategyType.BROADCAST; - } else if (Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH.equals(shipStrategy)) { - preSet2 = ShipStrategyType.PARTITION_HASH; - } else if (Optimizer.HINT_SHIP_STRATEGY_REPARTITION_RANGE.equals(shipStrategy)) { - preSet2 = ShipStrategyType.PARTITION_RANGE; - } else if (shipStrategy.equalsIgnoreCase(Optimizer.HINT_SHIP_STRATEGY_REPARTITION)) { - preSet2 = ShipStrategyType.PARTITION_RANDOM; - } else { - throw new CompilerException( - "Unknown hint for shipping strategy of input two: " + shipStrategy); - } - } - - // get the predecessors - DualInputOperator contr = getOperator(); - - Operator leftPred = contr.getFirstInput(); - Operator rightPred = contr.getSecondInput(); - - OptimizerNode pred1; - DagConnection conn1; - if (leftPred == null) { - throw new CompilerException( - "Error: Node for '" - + getOperator().getName() - + "' has no input set for first input."); - } else { - pred1 = contractToNode.get(leftPred); - conn1 = new DagConnection(pred1, this, defaultExecutionMode); - if (preSet1 != null) { - conn1.setShipStrategy(preSet1); - } - } - - // create the connection and add it - this.input1 = conn1; - pred1.addOutgoingConnection(conn1); - - OptimizerNode pred2; - DagConnection conn2; - if (rightPred == null) { - throw new CompilerException( - "Error: Node for '" - + getOperator().getName() - + "' has no input set for second input."); - } else { - pred2 = contractToNode.get(rightPred); - conn2 = new DagConnection(pred2, this, defaultExecutionMode); - if (preSet2 != null) { - conn2.setShipStrategy(preSet2); - } - } - - // create the connection and add it - this.input2 = conn2; - pred2.addOutgoingConnection(conn2); - } - - protected abstract List getPossibleProperties(); - - private List getProperties() { - if (this.cachedDescriptors == null) { - this.cachedDescriptors = getPossibleProperties(); - } - return this.cachedDescriptors; - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - // get what we inherit and what is preserved by our user code - final InterestingProperties props1 = - getInterestingProperties().filterByCodeAnnotations(this, 0); - final InterestingProperties props2 = - getInterestingProperties().filterByCodeAnnotations(this, 1); - - // add all properties relevant to this node - for (OperatorDescriptorDual dpd : getProperties()) { - for (GlobalPropertiesPair gp : dpd.getPossibleGlobalProperties()) { - // input 1 - props1.addGlobalProperties(gp.getProperties1()); - - // input 2 - props2.addGlobalProperties(gp.getProperties2()); - } - for (LocalPropertiesPair lp : dpd.getPossibleLocalProperties()) { - // input 1 - props1.addLocalProperties(lp.getProperties1()); - - // input 2 - props2.addLocalProperties(lp.getProperties2()); - } - } - this.input1.setInterestingProperties(props1); - this.input2.setInterestingProperties(props2); - - for (DagConnection conn : getBroadcastConnections()) { - conn.setInterestingProperties(new InterestingProperties()); - } - } - - @Override - public List getAlternativePlans(CostEstimator estimator) { - // check if we have a cached version - if (this.cachedPlans != null) { - return this.cachedPlans; - } - - boolean childrenSkippedDueToReplicatedInput = false; - - // step down to all producer nodes and calculate alternative plans - final List subPlans1 = - getFirstPredecessorNode().getAlternativePlans(estimator); - final List subPlans2 = - getSecondPredecessorNode().getAlternativePlans(estimator); - - // calculate alternative sub-plans for predecessor - final Set intGlobal1 = - this.input1.getInterestingProperties().getGlobalProperties(); - final Set intGlobal2 = - this.input2.getInterestingProperties().getGlobalProperties(); - - // calculate alternative sub-plans for broadcast inputs - final List> broadcastPlanChannels = - new ArrayList>(); - List broadcastConnections = getBroadcastConnections(); - List broadcastConnectionNames = getBroadcastConnectionNames(); - - for (int i = 0; i < broadcastConnections.size(); i++) { - DagConnection broadcastConnection = broadcastConnections.get(i); - String broadcastConnectionName = broadcastConnectionNames.get(i); - List broadcastPlanCandidates = - broadcastConnection.getSource().getAlternativePlans(estimator); - - // wrap the plan candidates in named channels - HashSet broadcastChannels = - new HashSet(broadcastPlanCandidates.size()); - for (PlanNode plan : broadcastPlanCandidates) { - final NamedChannel c = new NamedChannel(broadcastConnectionName, plan); - DataExchangeMode exMode = - DataExchangeMode.select( - broadcastConnection.getDataExchangeMode(), - ShipStrategyType.BROADCAST, - broadcastConnection.isBreakingPipeline()); - c.setShipStrategy(ShipStrategyType.BROADCAST, exMode); - broadcastChannels.add(c); - } - broadcastPlanChannels.add(broadcastChannels); - } - - final GlobalPropertiesPair[] allGlobalPairs; - final LocalPropertiesPair[] allLocalPairs; - { - Set pairsGlob = new HashSet(); - Set pairsLoc = new HashSet(); - for (OperatorDescriptorDual ods : getProperties()) { - pairsGlob.addAll(ods.getPossibleGlobalProperties()); - pairsLoc.addAll(ods.getPossibleLocalProperties()); - } - allGlobalPairs = pairsGlob.toArray(new GlobalPropertiesPair[pairsGlob.size()]); - allLocalPairs = pairsLoc.toArray(new LocalPropertiesPair[pairsLoc.size()]); - } - - final ArrayList outputPlans = new ArrayList(); - - final ExecutionMode input1Mode = this.input1.getDataExchangeMode(); - final ExecutionMode input2Mode = this.input2.getDataExchangeMode(); - - final int parallelism = getParallelism(); - final int inParallelism1 = getFirstPredecessorNode().getParallelism(); - final int inParallelism2 = getSecondPredecessorNode().getParallelism(); - - final boolean dopChange1 = parallelism != inParallelism1; - final boolean dopChange2 = parallelism != inParallelism2; - - final boolean input1breaksPipeline = this.input1.isBreakingPipeline(); - final boolean input2breaksPipeline = this.input2.isBreakingPipeline(); - - // enumerate all pairwise combination of the children's plans together with - // all possible operator strategy combination - - // create all candidates - for (PlanNode child1 : subPlans1) { - - if (child1.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if parallelism is not changed - if (dopChange1) { - // can not continue with this child - childrenSkippedDueToReplicatedInput = true; - continue; - } else { - this.input1.setShipStrategy(ShipStrategyType.FORWARD); - } - } - - for (PlanNode child2 : subPlans2) { - - if (child2.getGlobalProperties().isFullyReplicated()) { - // fully replicated input is always locally forwarded if parallelism is not - // changed - if (dopChange2) { - // can not continue with this child - childrenSkippedDueToReplicatedInput = true; - continue; - } else { - this.input2.setShipStrategy(ShipStrategyType.FORWARD); - } - } - - // check that the children go together. that is the case if they build upon the same - // candidate at the joined branch plan. - if (!areBranchCompatible(child1, child2)) { - continue; - } - - for (RequestedGlobalProperties igps1 : intGlobal1) { - // create a candidate channel for the first input. mark it cached, if the - // connection says so - final Channel c1 = new Channel(child1, this.input1.getMaterializationMode()); - if (this.input1.getShipStrategy() == null) { - // free to choose the ship strategy - igps1.parameterizeChannel(c1, dopChange1, input1Mode, input1breaksPipeline); - - // if the parallelism changed, make sure that we cancel out properties, - // unless the - // ship strategy preserves/establishes them even under changing parallelisms - if (dopChange1 && !c1.getShipStrategy().isNetworkStrategy()) { - c1.getGlobalProperties().reset(); - } - } else { - // ship strategy fixed by compiler hint - ShipStrategyType shipType = this.input1.getShipStrategy(); - DataExchangeMode exMode = - DataExchangeMode.select(input1Mode, shipType, input1breaksPipeline); - if (this.keys1 != null) { - c1.setShipStrategy(shipType, this.keys1.toFieldList(), exMode); - } else { - c1.setShipStrategy(shipType, exMode); - } - - if (dopChange1) { - c1.adjustGlobalPropertiesForFullParallelismChange(); - } - } - - for (RequestedGlobalProperties igps2 : intGlobal2) { - // create a candidate channel for the second input. mark it cached, if the - // connection says so - final Channel c2 = - new Channel(child2, this.input2.getMaterializationMode()); - if (this.input2.getShipStrategy() == null) { - // free to choose the ship strategy - igps2.parameterizeChannel( - c2, dopChange2, input2Mode, input2breaksPipeline); - - // if the parallelism changed, make sure that we cancel out properties, - // unless the - // ship strategy preserves/establishes them even under changing - // parallelisms - if (dopChange2 && !c2.getShipStrategy().isNetworkStrategy()) { - c2.getGlobalProperties().reset(); - } - } else { - // ship strategy fixed by compiler hint - ShipStrategyType shipType = this.input2.getShipStrategy(); - DataExchangeMode exMode = - DataExchangeMode.select( - input2Mode, shipType, input2breaksPipeline); - if (this.keys2 != null) { - c2.setShipStrategy(shipType, this.keys2.toFieldList(), exMode); - } else { - c2.setShipStrategy(shipType, exMode); - } - - if (dopChange2) { - c2.adjustGlobalPropertiesForFullParallelismChange(); - } - } - - /* ******************************************************************** - * NOTE: Depending on how we proceed with different partitioning, - * we might at some point need a compatibility check between - * the pairs of global properties. - * *******************************************************************/ - - outer: - for (GlobalPropertiesPair gpp : allGlobalPairs) { - if (gpp.getProperties1().isMetBy(c1.getGlobalProperties()) - && gpp.getProperties2().isMetBy(c2.getGlobalProperties())) { - for (OperatorDescriptorDual desc : getProperties()) { - if (desc.areCompatible( - gpp.getProperties1(), - gpp.getProperties2(), - c1.getGlobalProperties(), - c2.getGlobalProperties())) { - Channel c1Clone = c1.clone(); - c1Clone.setRequiredGlobalProps(gpp.getProperties1()); - c2.setRequiredGlobalProps(gpp.getProperties2()); - - // we form a valid combination, so create the local - // candidates - // for this - addLocalCandidates( - c1Clone, - c2, - broadcastPlanChannels, - igps1, - igps2, - outputPlans, - allLocalPairs, - estimator); - break outer; - } - } - } - } - - // break the loop over input2's possible global properties, if the property - // is fixed via a hint. All the properties are overridden by the hint - // anyways, - // so we can stop after the first - if (this.input2.getShipStrategy() != null) { - break; - } - } - - // break the loop over input1's possible global properties, if the property - // is fixed via a hint. All the properties are overridden by the hint anyways, - // so we can stop after the first - if (this.input1.getShipStrategy() != null) { - break; - } - } - } - } - - if (outputPlans.isEmpty()) { - if (childrenSkippedDueToReplicatedInput) { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Invalid use of replicated input."); - } else { - throw new CompilerException( - "No plan meeting the requirements could be created @ " - + this - + ". Most likely reason: Too restrictive plan hints."); - } - } - - // cost and prune the plans - for (PlanNode node : outputPlans) { - estimator.costOperator(node); - } - prunePlanAlternatives(outputPlans); - outputPlans.trimToSize(); - - this.cachedPlans = outputPlans; - return outputPlans; - } - - protected void addLocalCandidates( - Channel template1, - Channel template2, - List> broadcastPlanChannels, - RequestedGlobalProperties rgps1, - RequestedGlobalProperties rgps2, - List target, - LocalPropertiesPair[] validLocalCombinations, - CostEstimator estimator) { - for (RequestedLocalProperties ilp1 : - this.input1.getInterestingProperties().getLocalProperties()) { - final Channel in1 = template1.clone(); - ilp1.parameterizeChannel(in1); - - for (RequestedLocalProperties ilp2 : - this.input2.getInterestingProperties().getLocalProperties()) { - final Channel in2 = template2.clone(); - ilp2.parameterizeChannel(in2); - - for (OperatorDescriptorDual dps : getProperties()) { - for (LocalPropertiesPair lpp : dps.getPossibleLocalProperties()) { - if (lpp.getProperties1().isMetBy(in1.getLocalProperties()) - && lpp.getProperties2().isMetBy(in2.getLocalProperties())) { - // valid combination - // for non trivial local properties, we need to check that they are co - // compatible - // (such as when some sort order is requested, that both are the same - // sort order - if (dps.areCoFulfilled( - lpp.getProperties1(), - lpp.getProperties2(), - in1.getLocalProperties(), - in2.getLocalProperties())) { - // copy, because setting required properties and instantiation may - // change the channels and should not affect prior candidates - Channel in1Copy = in1.clone(); - in1Copy.setRequiredLocalProps(lpp.getProperties1()); - - Channel in2Copy = in2.clone(); - in2Copy.setRequiredLocalProps(lpp.getProperties2()); - - // all right, co compatible - instantiate( - dps, - in1Copy, - in2Copy, - broadcastPlanChannels, - target, - estimator, - rgps1, - rgps2, - ilp1, - ilp2); - break; - } - // else cannot use this pair, fall through the loop and try the next one - } - } - } - } - } - } - - protected void instantiate( - OperatorDescriptorDual operator, - Channel in1, - Channel in2, - List> broadcastPlanChannels, - List target, - CostEstimator estimator, - RequestedGlobalProperties globPropsReq1, - RequestedGlobalProperties globPropsReq2, - RequestedLocalProperties locPropsReq1, - RequestedLocalProperties locPropsReq2) { - final PlanNode inputSource1 = in1.getSource(); - final PlanNode inputSource2 = in2.getSource(); - - for (List broadcastChannelsCombination : - Sets.cartesianProduct(broadcastPlanChannels)) { - - boolean validCombination = true; - - // check whether the broadcast inputs use the same plan candidate at the branching point - for (int i = 0; i < broadcastChannelsCombination.size(); i++) { - NamedChannel nc = broadcastChannelsCombination.get(i); - PlanNode bcSource = nc.getSource(); - - if (!(areBranchCompatible(bcSource, inputSource1) - || areBranchCompatible(bcSource, inputSource2))) { - validCombination = false; - break; - } - - // check branch compatibility against all other broadcast variables - for (int k = 0; k < i; k++) { - PlanNode otherBcSource = broadcastChannelsCombination.get(k).getSource(); - - if (!areBranchCompatible(bcSource, otherBcSource)) { - validCombination = false; - break; - } - } - } - - if (!validCombination) { - continue; - } - - placePipelineBreakersIfNecessary(operator.getStrategy(), in1, in2); - - DualInputPlanNode node = operator.instantiate(in1, in2, this); - node.setBroadcastInputs(broadcastChannelsCombination); - - SemanticProperties semPropsGlobalPropFiltering = - getSemanticPropertiesForGlobalPropertyFiltering(); - GlobalProperties gp1 = - in1.getGlobalProperties() - .clone() - .filterBySemanticProperties(semPropsGlobalPropFiltering, 0); - GlobalProperties gp2 = - in2.getGlobalProperties() - .clone() - .filterBySemanticProperties(semPropsGlobalPropFiltering, 1); - GlobalProperties combined = operator.computeGlobalProperties(gp1, gp2); - - SemanticProperties semPropsLocalPropFiltering = - getSemanticPropertiesForLocalPropertyFiltering(); - LocalProperties lp1 = - in1.getLocalProperties() - .clone() - .filterBySemanticProperties(semPropsLocalPropFiltering, 0); - LocalProperties lp2 = - in2.getLocalProperties() - .clone() - .filterBySemanticProperties(semPropsLocalPropFiltering, 1); - LocalProperties locals = operator.computeLocalProperties(lp1, lp2); - - node.initProperties(combined, locals); - node.updatePropertiesWithUniqueSets(getUniqueFields()); - target.add(node); - } - } - - protected void placePipelineBreakersIfNecessary( - DriverStrategy strategy, Channel in1, Channel in2) { - // before we instantiate, check for deadlocks by tracing back to the open branches and - // checking - // whether either no input, or all of them have a dam - if (in1.isOnDynamicPath() - && in2.isOnDynamicPath() - && this.hereJoinedBranches != null - && this.hereJoinedBranches.size() > 0) { - boolean someDamOnLeftPaths = false; - boolean damOnAllLeftPaths = true; - boolean someDamOnRightPaths = false; - boolean damOnAllRightPaths = true; - - if (strategy.firstDam() == DamBehavior.FULL_DAM - || in1.getLocalStrategy().dams() - || in1.getTempMode().breaksPipeline()) { - someDamOnLeftPaths = true; - } else { - for (OptimizerNode brancher : this.hereJoinedBranches) { - PlanNode candAtBrancher = in1.getSource().getCandidateAtBranchPoint(brancher); - - // not all candidates are found, because this list includes joined branched from - // both regular inputs and broadcast vars - if (candAtBrancher == null) { - continue; - } - - SourceAndDamReport res = in1.getSource().hasDamOnPathDownTo(candAtBrancher); - if (res == NOT_FOUND) { - throw new CompilerException( - "Bug: Tracing dams for deadlock detection is broken."); - } else if (res == FOUND_SOURCE) { - damOnAllLeftPaths = false; - } else if (res == FOUND_SOURCE_AND_DAM) { - someDamOnLeftPaths = true; - } else { - throw new CompilerException(); - } - } - } - - if (strategy.secondDam() == DamBehavior.FULL_DAM - || in2.getLocalStrategy().dams() - || in2.getTempMode().breaksPipeline()) { - someDamOnRightPaths = true; - } else { - for (OptimizerNode brancher : this.hereJoinedBranches) { - PlanNode candAtBrancher = in2.getSource().getCandidateAtBranchPoint(brancher); - - // not all candidates are found, because this list includes joined branched from - // both regular inputs and broadcast vars - if (candAtBrancher == null) { - continue; - } - - SourceAndDamReport res = in2.getSource().hasDamOnPathDownTo(candAtBrancher); - if (res == NOT_FOUND) { - throw new CompilerException( - "Bug: Tracing dams for deadlock detection is broken."); - } else if (res == FOUND_SOURCE) { - damOnAllRightPaths = false; - } else if (res == FOUND_SOURCE_AND_DAM) { - someDamOnRightPaths = true; - } else { - throw new CompilerException(); - } - } - } - - // okay combinations are both all dam or both no dam - if ((damOnAllLeftPaths && damOnAllRightPaths) - || (!someDamOnLeftPaths && !someDamOnRightPaths)) { - // good, either both materialize already on the way, or both fully pipeline - } else { - if (someDamOnLeftPaths && !damOnAllRightPaths) { - // right needs a pipeline breaker - in2.setTempMode(in2.getTempMode().makePipelineBreaker()); - } - - if (someDamOnRightPaths && !damOnAllLeftPaths) { - // right needs a pipeline breaker - in1.setTempMode(in1.getTempMode().makePipelineBreaker()); - } - } - } - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - // handle the data flow branching for the regular inputs - addClosedBranches(getFirstPredecessorNode().closedBranchingNodes); - addClosedBranches(getSecondPredecessorNode().closedBranchingNodes); - - List result1 = - getFirstPredecessorNode().getBranchesForParent(getFirstIncomingConnection()); - List result2 = - getSecondPredecessorNode().getBranchesForParent(getSecondIncomingConnection()); - - ArrayList inputsMerged = - new ArrayList(); - mergeLists(result1, result2, inputsMerged, true); - - // handle the data flow branching for the broadcast inputs - List result = - computeUnclosedBranchStackForBroadcastInputs(inputsMerged); - - this.openBranches = - (result == null || result.isEmpty()) - ? Collections.emptyList() - : result; - } - - @Override - public SemanticProperties getSemanticProperties() { - return getOperator().getSemanticProperties(); - } - - protected SemanticProperties getSemanticPropertiesForLocalPropertyFiltering() { - return this.getSemanticProperties(); - } - - protected SemanticProperties getSemanticPropertiesForGlobalPropertyFiltering() { - return this.getSemanticProperties(); - } - - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - if (this.input1 == null || this.input2 == null) { - throw new CompilerException(); - } - - getFirstPredecessorNode().accept(visitor); - getSecondPredecessorNode().accept(visitor); - - for (DagConnection connection : getBroadcastConnections()) { - connection.getSource().accept(visitor); - } - - visitor.postVisit(this); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/UnaryOperatorNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/UnaryOperatorNode.java deleted file mode 100644 index 9c2d7d7b1f77a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/UnaryOperatorNode.java +++ /dev/null @@ -1,77 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.operators.OperatorDescriptorSingle; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class UnaryOperatorNode extends SingleInputNode { - - private final List operators; - - private final String name; - - public UnaryOperatorNode( - String name, SingleInputOperator operator, boolean onDynamicPath) { - super(operator); - - this.name = name; - this.operators = new ArrayList<>(); - this.onDynamicPath = onDynamicPath; - } - - public UnaryOperatorNode(String name, FieldSet keys, OperatorDescriptorSingle... operators) { - this(name, keys, Arrays.asList(operators)); - } - - public UnaryOperatorNode(String name, FieldSet keys, List operators) { - super(keys); - - this.operators = operators; - this.name = name; - } - - @Override - protected List getPossibleProperties() { - return this.operators; - } - - @Override - public String getOperatorName() { - return this.name; - } - - @Override - public SemanticProperties getSemanticProperties() { - return new SingleInputSemanticProperties.AllFieldsForwardedProperties(); - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // we have no estimates by default - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java deleted file mode 100644 index 8ad9e117c4e0b..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java +++ /dev/null @@ -1,663 +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.optimizer.dag; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties; -import org.apache.flink.api.common.operators.base.DeltaIterationBase; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeinfo.NothingTypeInfo; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.InterestingProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.operators.OperatorDescriptorDual; -import org.apache.flink.optimizer.operators.SolutionSetDeltaOperator; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SolutionSetPlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plan.WorksetPlanNode; -import org.apache.flink.optimizer.traversals.InterestingPropertyVisitor; -import org.apache.flink.optimizer.util.NoOpBinaryUdfOp; -import org.apache.flink.optimizer.util.NoOpUnaryUdfOp; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.types.Nothing; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -/** A node in the optimizer's program representation for a workset iteration. */ -public class WorksetIterationNode extends TwoInputNode implements IterationNode { - - private static final int DEFAULT_COST_WEIGHT = 20; - - private final FieldList solutionSetKeyFields; - - private final GlobalProperties partitionedProperties; - - private final List dataProperties; - - private SolutionSetNode solutionSetNode; - - private WorksetNode worksetNode; - - private OptimizerNode solutionSetDelta; - - private OptimizerNode nextWorkset; - - private DagConnection solutionSetDeltaRootConnection; - - private DagConnection nextWorksetRootConnection; - - private SingleRootJoiner singleRoot; - - private boolean solutionDeltaImmediatelyAfterSolutionJoin; - - private final int costWeight; - - // -------------------------------------------------------------------------------------------- - - /** - * Creates a new node with a single input for the optimizer plan. - * - * @param iteration The iteration operator that the node represents. - */ - public WorksetIterationNode(DeltaIterationBase iteration) { - super(iteration); - - final int[] ssKeys = iteration.getSolutionSetKeyFields(); - if (ssKeys == null || ssKeys.length == 0) { - throw new CompilerException( - "Invalid WorksetIteration: No key fields defined for the solution set."); - } - this.solutionSetKeyFields = new FieldList(ssKeys); - this.partitionedProperties = new GlobalProperties(); - this.partitionedProperties.setHashPartitioned(this.solutionSetKeyFields); - - int weight = - iteration.getMaximumNumberOfIterations() > 0 - ? iteration.getMaximumNumberOfIterations() - : DEFAULT_COST_WEIGHT; - - if (weight > OptimizerNode.MAX_DYNAMIC_PATH_COST_WEIGHT) { - weight = OptimizerNode.MAX_DYNAMIC_PATH_COST_WEIGHT; - } - this.costWeight = weight; - - this.dataProperties = - Collections.singletonList( - new WorksetOpDescriptor(this.solutionSetKeyFields)); - } - - // -------------------------------------------------------------------------------------------- - - public DeltaIterationBase getIterationContract() { - return (DeltaIterationBase) getOperator(); - } - - public SolutionSetNode getSolutionSetNode() { - return this.solutionSetNode; - } - - public WorksetNode getWorksetNode() { - return this.worksetNode; - } - - public OptimizerNode getNextWorkset() { - return this.nextWorkset; - } - - public OptimizerNode getSolutionSetDelta() { - return this.solutionSetDelta; - } - - public void setPartialSolution(SolutionSetNode solutionSetNode, WorksetNode worksetNode) { - if (this.solutionSetNode != null || this.worksetNode != null) { - throw new IllegalStateException( - "Error: Initializing WorksetIterationNode multiple times."); - } - this.solutionSetNode = solutionSetNode; - this.worksetNode = worksetNode; - } - - public void setNextPartialSolution( - OptimizerNode solutionSetDelta, - OptimizerNode nextWorkset, - ExecutionMode executionMode) { - - // check whether the next partial solution is itself the join with - // the partial solution (so we can potentially do direct updates) - if (solutionSetDelta instanceof TwoInputNode) { - TwoInputNode solutionDeltaTwoInput = (TwoInputNode) solutionSetDelta; - if (solutionDeltaTwoInput.getFirstPredecessorNode() == this.solutionSetNode - || solutionDeltaTwoInput.getSecondPredecessorNode() == this.solutionSetNode) { - this.solutionDeltaImmediatelyAfterSolutionJoin = true; - } - } - - // there needs to be at least one node in the workset path, so - // if the next workset is equal to the workset, we need to inject a no-op node - if (nextWorkset == worksetNode || nextWorkset instanceof BinaryUnionNode) { - NoOpNode noop = new NoOpNode(); - noop.setParallelism(getParallelism()); - - DagConnection noOpConn = new DagConnection(nextWorkset, noop, executionMode); - noop.setIncomingConnection(noOpConn); - nextWorkset.addOutgoingConnection(noOpConn); - - nextWorkset = noop; - } - - // attach an extra node to the solution set delta for the cases where we need to repartition - UnaryOperatorNode solutionSetDeltaUpdateAux = - new UnaryOperatorNode( - "Solution-Set Delta", - getSolutionSetKeyFields(), - new SolutionSetDeltaOperator(getSolutionSetKeyFields())); - solutionSetDeltaUpdateAux.setParallelism(getParallelism()); - - DagConnection conn = - new DagConnection(solutionSetDelta, solutionSetDeltaUpdateAux, executionMode); - solutionSetDeltaUpdateAux.setIncomingConnection(conn); - solutionSetDelta.addOutgoingConnection(conn); - - this.solutionSetDelta = solutionSetDeltaUpdateAux; - this.nextWorkset = nextWorkset; - - this.singleRoot = new SingleRootJoiner(); - this.solutionSetDeltaRootConnection = - new DagConnection(solutionSetDeltaUpdateAux, this.singleRoot, executionMode); - - this.nextWorksetRootConnection = - new DagConnection(nextWorkset, this.singleRoot, executionMode); - this.singleRoot.setInputs( - this.solutionSetDeltaRootConnection, this.nextWorksetRootConnection); - - solutionSetDeltaUpdateAux.addOutgoingConnection(this.solutionSetDeltaRootConnection); - nextWorkset.addOutgoingConnection(this.nextWorksetRootConnection); - } - - public int getCostWeight() { - return this.costWeight; - } - - public TwoInputNode getSingleRootOfStepFunction() { - return this.singleRoot; - } - - public FieldList getSolutionSetKeyFields() { - return this.solutionSetKeyFields; - } - - public OptimizerNode getInitialSolutionSetPredecessorNode() { - return getFirstPredecessorNode(); - } - - public OptimizerNode getInitialWorksetPredecessorNode() { - return getSecondPredecessorNode(); - } - - // -------------------------------------------------------------------------------------------- - - @Override - public String getOperatorName() { - return "Workset Iteration"; - } - - @Override - public SemanticProperties getSemanticProperties() { - return new EmptySemanticProperties(); - } - - protected void readStubAnnotations() {} - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - this.estimatedOutputSize = getFirstPredecessorNode().getEstimatedOutputSize(); - this.estimatedNumRecords = getFirstPredecessorNode().getEstimatedNumRecords(); - } - - // -------------------------------------------------------------------------------------------- - // Properties and Optimization - // -------------------------------------------------------------------------------------------- - - @Override - protected List getPossibleProperties() { - return this.dataProperties; - } - - @Override - public void computeInterestingPropertiesForInputs(CostEstimator estimator) { - // our own solution (the solution set) is always partitioned and this cannot be adjusted - // depending on what the successor to the workset iteration requests. for that reason, - // we ignore incoming interesting properties. - - // in addition, we need to make 2 interesting property passes, because the root of the step - // function - // that computes the next workset needs the interesting properties as generated by the - // workset source of the step function. the second pass concerns only the workset path. - // as initial interesting properties, we have the trivial ones for the step function, - // and partitioned on the solution set key for the solution set delta - - RequestedGlobalProperties partitionedProperties = new RequestedGlobalProperties(); - partitionedProperties.setHashPartitioned(this.solutionSetKeyFields); - InterestingProperties partitionedIP = new InterestingProperties(); - partitionedIP.addGlobalProperties(partitionedProperties); - partitionedIP.addLocalProperties(new RequestedLocalProperties()); - - this.nextWorksetRootConnection.setInterestingProperties(new InterestingProperties()); - this.solutionSetDeltaRootConnection.setInterestingProperties(partitionedIP.clone()); - - InterestingPropertyVisitor ipv = new InterestingPropertyVisitor(estimator); - this.nextWorkset.accept(ipv); - this.solutionSetDelta.accept(ipv); - - // take the interesting properties of the partial solution and add them to the root - // interesting properties - InterestingProperties worksetIntProps = this.worksetNode.getInterestingProperties(); - InterestingProperties intProps = new InterestingProperties(); - intProps.getGlobalProperties().addAll(worksetIntProps.getGlobalProperties()); - intProps.getLocalProperties().addAll(worksetIntProps.getLocalProperties()); - - // clear all interesting properties to prepare the second traversal - this.nextWorksetRootConnection.clearInterestingProperties(); - this.nextWorkset.accept(InterestingPropertiesClearer.INSTANCE); - - // 2nd pass - this.nextWorksetRootConnection.setInterestingProperties(intProps); - this.nextWorkset.accept(ipv); - - // now add the interesting properties of the workset to the workset input - final InterestingProperties inProps = this.worksetNode.getInterestingProperties().clone(); - inProps.addGlobalProperties(new RequestedGlobalProperties()); - inProps.addLocalProperties(new RequestedLocalProperties()); - this.input2.setInterestingProperties(inProps); - - // the partial solution must be hash partitioned, so it has only that as interesting - // properties - this.input1.setInterestingProperties(partitionedIP); - } - - @Override - public void clearInterestingProperties() { - super.clearInterestingProperties(); - - this.nextWorksetRootConnection.clearInterestingProperties(); - this.solutionSetDeltaRootConnection.clearInterestingProperties(); - - this.nextWorkset.accept(InterestingPropertiesClearer.INSTANCE); - this.solutionSetDelta.accept(InterestingPropertiesClearer.INSTANCE); - } - - @SuppressWarnings("unchecked") - @Override - protected void instantiate( - OperatorDescriptorDual operator, - Channel solutionSetIn, - Channel worksetIn, - List> broadcastPlanChannels, - List target, - CostEstimator estimator, - RequestedGlobalProperties globPropsReqSolutionSet, - RequestedGlobalProperties globPropsReqWorkset, - RequestedLocalProperties locPropsReqSolutionSet, - RequestedLocalProperties locPropsReqWorkset) { - // check for pipeline breaking using hash join with build on the solution set side - placePipelineBreakersIfNecessary( - DriverStrategy.HYBRIDHASH_BUILD_FIRST, solutionSetIn, worksetIn); - - // NOTES ON THE ENUMERATION OF THE STEP FUNCTION PLANS: - // Whenever we instantiate the iteration, we enumerate new candidates for the step function. - // That way, we make sure we have an appropriate plan for each candidate for the initial - // partial solution, - // we have a fitting candidate for the step function (often, work is pushed out of the step - // function). - // Among the candidates of the step function, we keep only those that meet the requested - // properties of the - // current candidate initial partial solution. That makes sure these properties exist at the - // beginning of - // every iteration. - - // 1) Because we enumerate multiple times, we may need to clean the cached plans - // before starting another enumeration - this.nextWorkset.accept(PlanCacheCleaner.INSTANCE); - this.solutionSetDelta.accept(PlanCacheCleaner.INSTANCE); - - // 2) Give the partial solution the properties of the current candidate for the initial - // partial solution - // This concerns currently only the workset. - this.worksetNode.setCandidateProperties( - worksetIn.getGlobalProperties(), worksetIn.getLocalProperties(), worksetIn); - this.solutionSetNode.setCandidateProperties( - this.partitionedProperties, new LocalProperties(), solutionSetIn); - - final SolutionSetPlanNode sspn = this.solutionSetNode.getCurrentSolutionSetPlanNode(); - final WorksetPlanNode wspn = this.worksetNode.getCurrentWorksetPlanNode(); - - // 3) Get the alternative plans - List solutionSetDeltaCandidates = - this.solutionSetDelta.getAlternativePlans(estimator); - List worksetCandidates = this.nextWorkset.getAlternativePlans(estimator); - - // 4) Throw away all that are not compatible with the properties currently requested to the - // initial partial solution - - // Make sure that the workset candidates fulfill the input requirements - { - List newCandidates = new ArrayList(); - - for (Iterator planDeleter = worksetCandidates.iterator(); - planDeleter.hasNext(); ) { - PlanNode candidate = planDeleter.next(); - - GlobalProperties atEndGlobal = candidate.getGlobalProperties(); - LocalProperties atEndLocal = candidate.getLocalProperties(); - - FeedbackPropertiesMeetRequirementsReport report = - candidate.checkPartialSolutionPropertiesMet(wspn, atEndGlobal, atEndLocal); - - if (report == FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION) { - // depends only through broadcast variable on the workset solution - } else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) { - // attach a no-op node through which we create the properties of the original - // input - Channel toNoOp = new Channel(candidate); - globPropsReqWorkset.parameterizeChannel( - toNoOp, false, nextWorksetRootConnection.getDataExchangeMode(), false); - locPropsReqWorkset.parameterizeChannel(toNoOp); - - NoOpUnaryUdfOp noOpUnaryUdfOp = new NoOpUnaryUdfOp<>(); - noOpUnaryUdfOp.setInput(candidate.getProgramOperator()); - - UnaryOperatorNode rebuildWorksetPropertiesNode = - new UnaryOperatorNode( - "Rebuild Workset Properties", noOpUnaryUdfOp, true); - - rebuildWorksetPropertiesNode.setParallelism(candidate.getParallelism()); - - SingleInputPlanNode rebuildWorksetPropertiesPlanNode = - new SingleInputPlanNode( - rebuildWorksetPropertiesNode, - "Rebuild Workset Properties", - toNoOp, - DriverStrategy.UNARY_NO_OP); - rebuildWorksetPropertiesPlanNode.initProperties( - toNoOp.getGlobalProperties(), toNoOp.getLocalProperties()); - estimator.costOperator(rebuildWorksetPropertiesPlanNode); - - GlobalProperties atEndGlobalModified = - rebuildWorksetPropertiesPlanNode.getGlobalProperties(); - LocalProperties atEndLocalModified = - rebuildWorksetPropertiesPlanNode.getLocalProperties(); - - if (!(atEndGlobalModified.equals(atEndGlobal) - && atEndLocalModified.equals(atEndLocal))) { - FeedbackPropertiesMeetRequirementsReport report2 = - candidate.checkPartialSolutionPropertiesMet( - wspn, atEndGlobalModified, atEndLocalModified); - if (report2 != FeedbackPropertiesMeetRequirementsReport.NOT_MET) { - newCandidates.add(rebuildWorksetPropertiesPlanNode); - } - } - - // remove the original operator and add the modified candidate - planDeleter.remove(); - } - } - - worksetCandidates.addAll(newCandidates); - } - - if (worksetCandidates.isEmpty()) { - return; - } - - // sanity check the solution set delta - for (PlanNode solutionSetDeltaCandidate : solutionSetDeltaCandidates) { - SingleInputPlanNode candidate = (SingleInputPlanNode) solutionSetDeltaCandidate; - GlobalProperties gp = candidate.getGlobalProperties(); - - if (gp.getPartitioning() != PartitioningProperty.HASH_PARTITIONED - || gp.getPartitioningFields() == null - || !gp.getPartitioningFields().equals(this.solutionSetKeyFields)) { - throw new CompilerException("Bug: The solution set delta is not partitioned."); - } - } - - // 5) Create a candidate for the Iteration Node for every remaining plan of the step - // function. - - final GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(this.solutionSetKeyFields); - gp.addUniqueFieldCombination(this.solutionSetKeyFields); - - LocalProperties lp = LocalProperties.EMPTY.addUniqueFields(this.solutionSetKeyFields); - - // take all combinations of solution set delta and workset plans - for (PlanNode worksetCandidate : worksetCandidates) { - for (PlanNode solutionSetCandidate : solutionSetDeltaCandidates) { - // check whether they have the same operator at their latest branching point - if (this.singleRoot.areBranchCompatible(solutionSetCandidate, worksetCandidate)) { - - SingleInputPlanNode siSolutionDeltaCandidate = - (SingleInputPlanNode) solutionSetCandidate; - boolean immediateDeltaUpdate; - - // check whether we need a dedicated solution set delta operator, or whether we - // can update on the fly - if (siSolutionDeltaCandidate.getInput().getShipStrategy() - == ShipStrategyType.FORWARD - && this.solutionDeltaImmediatelyAfterSolutionJoin) { - // we do not need this extra node. we can make the predecessor the delta - // sanity check the node and connection - if (siSolutionDeltaCandidate.getDriverStrategy() - != DriverStrategy.UNARY_NO_OP - || siSolutionDeltaCandidate.getInput().getLocalStrategy() - != LocalStrategy.NONE) { - throw new CompilerException("Invalid Solution set delta node."); - } - - solutionSetCandidate = siSolutionDeltaCandidate.getInput().getSource(); - immediateDeltaUpdate = true; - } else { - // was not partitioned, we need to keep this node. - // mark that we materialize the input - siSolutionDeltaCandidate.getInput().setTempMode(TempMode.PIPELINE_BREAKER); - immediateDeltaUpdate = false; - } - - WorksetIterationPlanNode wsNode = - new WorksetIterationPlanNode( - this, - this.getOperator().getName(), - solutionSetIn, - worksetIn, - sspn, - wspn, - worksetCandidate, - solutionSetCandidate); - wsNode.setImmediateSolutionSetUpdate(immediateDeltaUpdate); - wsNode.initProperties(gp, lp); - target.add(wsNode); - } - } - } - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - // IMPORTANT: First compute closed branches from the two inputs - // we need to do this because the runtime iteration head effectively joins - addClosedBranches(getFirstPredecessorNode().closedBranchingNodes); - addClosedBranches(getSecondPredecessorNode().closedBranchingNodes); - - List result1 = - getFirstPredecessorNode().getBranchesForParent(getFirstIncomingConnection()); - List result2 = - getSecondPredecessorNode().getBranchesForParent(getSecondIncomingConnection()); - - ArrayList inputsMerged1 = - new ArrayList(); - mergeLists( - result1, - result2, - inputsMerged1, - true); // this method also sets which branches are joined here (in the head) - - addClosedBranches(getSingleRootOfStepFunction().closedBranchingNodes); - - ArrayList inputsMerged2 = - new ArrayList(); - List result3 = getSingleRootOfStepFunction().openBranches; - mergeLists(inputsMerged1, result3, inputsMerged2, true); - - // handle the data flow branching for the broadcast inputs - List result = - computeUnclosedBranchStackForBroadcastInputs(inputsMerged2); - - this.openBranches = - (result == null || result.isEmpty()) - ? Collections.emptyList() - : result; - } - - // -------------------------------------------------------------------------------------------- - // Iteration Specific Traversals - // -------------------------------------------------------------------------------------------- - - public void acceptForStepFunction(Visitor visitor) { - this.singleRoot.accept(visitor); - } - - // -------------------------------------------------------------------------------------------- - // Utility Classes - // -------------------------------------------------------------------------------------------- - - private static final class WorksetOpDescriptor extends OperatorDescriptorDual { - - private WorksetOpDescriptor(FieldList solutionSetKeys) { - super(solutionSetKeys, null); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.NONE; - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties partitionedGp = new RequestedGlobalProperties(); - partitionedGp.setHashPartitioned(this.keys1); - return Collections.singletonList( - new GlobalPropertiesPair(partitionedGp, new RequestedGlobalProperties())); - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - return true; - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - throw new UnsupportedOperationException(); - } - - @Override - public GlobalProperties computeGlobalProperties( - GlobalProperties in1, GlobalProperties in2) { - throw new UnsupportedOperationException(); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - throw new UnsupportedOperationException(); - } - } - - public static class SingleRootJoiner extends TwoInputNode { - - SingleRootJoiner() { - super(new NoOpBinaryUdfOp(new NothingTypeInfo())); - - setParallelism(1); - } - - public void setInputs(DagConnection input1, DagConnection input2) { - this.input1 = input1; - this.input2 = input2; - } - - @Override - public String getOperatorName() { - return "Internal Utility Node"; - } - - @Override - protected List getPossibleProperties() { - return Collections.emptyList(); - } - - @Override - protected void computeOperatorSpecificDefaultEstimates(DataStatistics statistics) { - // no estimates are needed here - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetNode.java deleted file mode 100644 index f81d188a045ed..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetNode.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.optimizer.dag; - -import org.apache.flink.api.common.operators.base.DeltaIterationBase.WorksetPlaceHolder; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.WorksetPlanNode; - -import java.util.Collections; -import java.util.List; - -/** - * The optimizer's internal representation of the partial solution that is input to a bulk - * iteration. - */ -public class WorksetNode extends AbstractPartialSolutionNode { - - private final WorksetIterationNode iterationNode; - - public WorksetNode(WorksetPlaceHolder psph, WorksetIterationNode iterationNode) { - super(psph); - this.iterationNode = iterationNode; - } - - // -------------------------------------------------------------------------------------------- - - public void setCandidateProperties( - GlobalProperties gProps, LocalProperties lProps, Channel initialInput) { - if (this.cachedPlans != null) { - throw new IllegalStateException(); - } else { - WorksetPlanNode wspn = - new WorksetPlanNode( - this, - "Workset (" + this.getOperator().getName() + ")", - gProps, - lProps, - initialInput); - this.cachedPlans = Collections.singletonList(wspn); - } - } - - public WorksetPlanNode getCurrentWorksetPlanNode() { - if (this.cachedPlans != null) { - return (WorksetPlanNode) this.cachedPlans.get(0); - } else { - throw new IllegalStateException(); - } - } - - public WorksetIterationNode getIterationNode() { - return this.iterationNode; - } - - @Override - public void computeOutputEstimates(DataStatistics statistics) { - copyEstimates(this.iterationNode.getInitialWorksetPredecessorNode()); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the contract object for this data source node. - * - * @return The contract. - */ - @Override - public WorksetPlaceHolder getOperator() { - return (WorksetPlaceHolder) super.getOperator(); - } - - @Override - public String getOperatorName() { - return "Workset"; - } - - @Override - public void computeUnclosedBranchStack() { - if (this.openBranches != null) { - return; - } - - DagConnection worksetInput = this.iterationNode.getSecondIncomingConnection(); - OptimizerNode worksetSource = worksetInput.getSource(); - - addClosedBranches(worksetSource.closedBranchingNodes); - List fromInput = worksetSource.getBranchesForParent(worksetInput); - this.openBranches = - (fromInput == null || fromInput.isEmpty()) - ? Collections.emptyList() - : fromInput; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java deleted file mode 100644 index e19bced56d74f..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java +++ /dev/null @@ -1,559 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashSet; -import java.util.Set; - -/** - * This class represents global properties of the data at a certain point in the plan. Global - * properties are properties that describe data across different partitions, such as whether the - * data is hash partitioned, range partitioned, replicated, etc. - */ -public class GlobalProperties implements Cloneable { - - public static final Logger LOG = LoggerFactory.getLogger(GlobalProperties.class); - - private PartitioningProperty partitioning; // the type partitioning - - private FieldList partitioningFields; // the fields which are partitioned - - private Ordering ordering; // order of the partitioned fields, if it is an ordered (range) range - // partitioning - - private Set

uniqueFieldCombinations; - - private Partitioner customPartitioner; - - private DataDistribution distribution; - - // -------------------------------------------------------------------------------------------- - - /** Initializes the global properties with no partitioning. */ - public GlobalProperties() { - this.partitioning = PartitioningProperty.RANDOM_PARTITIONED; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Sets this global properties to represent a hash partitioning. - * - * @param partitionedFields The key fields on which the data is hash partitioned. - */ - public void setHashPartitioned(FieldList partitionedFields) { - if (partitionedFields == null) { - throw new NullPointerException(); - } - - this.partitioning = PartitioningProperty.HASH_PARTITIONED; - this.partitioningFields = partitionedFields; - this.ordering = null; - } - - /** - * Set the parameters for range partition. - * - * @param ordering Order of the partitioned fields - */ - public void setRangePartitioned(Ordering ordering) { - if (ordering == null) { - throw new NullPointerException(); - } - - this.partitioning = PartitioningProperty.RANGE_PARTITIONED; - this.ordering = ordering; - this.partitioningFields = ordering.getInvolvedIndexes(); - } - - /** - * Set the parameters for range partition. - * - * @param ordering Order of the partitioned fields - * @param distribution The data distribution for range partition. User can supply a customized - * data distribution, also the data distribution can be null. - */ - public void setRangePartitioned(Ordering ordering, DataDistribution distribution) { - if (ordering == null) { - throw new NullPointerException(); - } - - this.partitioning = PartitioningProperty.RANGE_PARTITIONED; - this.ordering = ordering; - this.partitioningFields = ordering.getInvolvedIndexes(); - this.distribution = distribution; - } - - public void setAnyPartitioning(FieldList partitionedFields) { - if (partitionedFields == null) { - throw new NullPointerException(); - } - - this.partitioning = PartitioningProperty.ANY_PARTITIONING; - this.partitioningFields = partitionedFields; - this.ordering = null; - } - - public void setRandomPartitioned() { - this.partitioning = PartitioningProperty.RANDOM_PARTITIONED; - this.partitioningFields = null; - this.ordering = null; - } - - public void setFullyReplicated() { - this.partitioning = PartitioningProperty.FULL_REPLICATION; - this.partitioningFields = null; - this.ordering = null; - } - - public void setForcedRebalanced() { - this.partitioning = PartitioningProperty.FORCED_REBALANCED; - this.partitioningFields = null; - this.ordering = null; - } - - public void setCustomPartitioned(FieldList partitionedFields, Partitioner partitioner) { - if (partitionedFields == null || partitioner == null) { - throw new NullPointerException(); - } - - this.partitioning = PartitioningProperty.CUSTOM_PARTITIONING; - this.partitioningFields = partitionedFields; - this.ordering = null; - this.customPartitioner = partitioner; - } - - public void addUniqueFieldCombination(FieldSet fields) { - if (fields == null) { - return; - } - if (this.uniqueFieldCombinations == null) { - this.uniqueFieldCombinations = new HashSet
(); - } - this.uniqueFieldCombinations.add(fields); - } - - public void clearUniqueFieldCombinations() { - if (this.uniqueFieldCombinations != null) { - this.uniqueFieldCombinations = null; - } - } - - public Set
getUniqueFieldCombination() { - return this.uniqueFieldCombinations; - } - - public FieldList getPartitioningFields() { - return this.partitioningFields; - } - - public Ordering getPartitioningOrdering() { - return this.ordering; - } - - public PartitioningProperty getPartitioning() { - return this.partitioning; - } - - public Partitioner getCustomPartitioner() { - return this.customPartitioner; - } - - public DataDistribution getDataDistribution() { - return this.distribution; - } - - // -------------------------------------------------------------------------------------------- - - public boolean isPartitionedOnFields(FieldSet fields) { - if (this.partitioning.isPartitionedOnKey() - && fields.isValidSubset(this.partitioningFields)) { - return true; - } else if (this.uniqueFieldCombinations != null) { - for (FieldSet set : this.uniqueFieldCombinations) { - if (fields.isValidSubset(set)) { - return true; - } - } - return false; - } else { - return false; - } - } - - public boolean isExactlyPartitionedOnFields(FieldList fields) { - return this.partitioning.isPartitionedOnKey() - && fields.isExactMatch(this.partitioningFields); - } - - public boolean matchesOrderedPartitioning(Ordering o) { - if (this.partitioning == PartitioningProperty.RANGE_PARTITIONED) { - if (this.ordering.getNumberOfFields() > o.getNumberOfFields()) { - return false; - } - - for (int i = 0; i < this.ordering.getNumberOfFields(); i++) { - if (!this.ordering.getFieldNumber(i).equals(o.getFieldNumber(i))) { - return false; - } - - // if this one request no order, everything is good - final Order oo = o.getOrder(i); - final Order to = this.ordering.getOrder(i); - if (oo != Order.NONE) { - if (oo == Order.ANY) { - // if any order is requested, any not NONE order is good - if (to == Order.NONE) { - return false; - } - } else if (oo != to) { - // the orders must be equal - return false; - } - } - } - return true; - } else { - return false; - } - } - - public boolean isFullyReplicated() { - return this.partitioning == PartitioningProperty.FULL_REPLICATION; - } - - /** Checks, if the properties in this object are trivial, i.e. only standard values. */ - public boolean isTrivial() { - return partitioning == PartitioningProperty.RANDOM_PARTITIONED; - } - - /** This method resets the properties to a state where no properties are given. */ - public void reset() { - this.partitioning = PartitioningProperty.RANDOM_PARTITIONED; - this.ordering = null; - this.partitioningFields = null; - } - - /** - * Filters these GlobalProperties by the fields that are forwarded to the output as described by - * the SemanticProperties. - * - * @param props The semantic properties holding information about forwarded fields. - * @param input The index of the input. - * @return The filtered GlobalProperties - */ - public GlobalProperties filterBySemanticProperties(SemanticProperties props, int input) { - - if (props == null) { - throw new NullPointerException("SemanticProperties may not be null."); - } - - GlobalProperties gp = new GlobalProperties(); - - // filter partitioning - switch (this.partitioning) { - case RANGE_PARTITIONED: - // check if ordering is preserved - Ordering newOrdering = new Ordering(); - for (int i = 0; i < this.ordering.getInvolvedIndexes().size(); i++) { - int sourceField = this.ordering.getInvolvedIndexes().get(i); - FieldSet targetField = props.getForwardingTargetFields(input, sourceField); - - if (targetField == null || targetField.size() == 0) { - // partitioning is destroyed - newOrdering = null; - break; - } else { - // use any field of target fields for now. We should use something like - // field equivalence sets in the future. - if (targetField.size() > 1) { - LOG.warn( - "Found that a field is forwarded to more than one target field in " - + "semantic forwarded field information. Will only use the field with the lowest index."); - } - newOrdering.appendOrdering( - targetField.toArray()[0], - this.ordering.getType(i), - this.ordering.getOrder(i)); - } - } - if (newOrdering != null) { - gp.partitioning = PartitioningProperty.RANGE_PARTITIONED; - gp.ordering = newOrdering; - gp.partitioningFields = newOrdering.getInvolvedIndexes(); - gp.distribution = this.distribution; - } - break; - case HASH_PARTITIONED: - case ANY_PARTITIONING: - case CUSTOM_PARTITIONING: - FieldList newPartitioningFields = new FieldList(); - for (int sourceField : this.partitioningFields) { - FieldSet targetField = props.getForwardingTargetFields(input, sourceField); - - if (targetField == null || targetField.size() == 0) { - newPartitioningFields = null; - break; - } else { - // use any field of target fields for now. We should use something like - // field equivalence sets in the future. - if (targetField.size() > 1) { - LOG.warn( - "Found that a field is forwarded to more than one target field in " - + "semantic forwarded field information. Will only use the field with the lowest index."); - } - newPartitioningFields = - newPartitioningFields.addField(targetField.toArray()[0]); - } - } - if (newPartitioningFields != null) { - gp.partitioning = this.partitioning; - gp.partitioningFields = newPartitioningFields; - gp.customPartitioner = this.customPartitioner; - } - break; - case FORCED_REBALANCED: - case FULL_REPLICATION: - case RANDOM_PARTITIONED: - gp.partitioning = this.partitioning; - break; - default: - throw new RuntimeException("Unknown partitioning type."); - } - - // filter unique field combinations - if (this.uniqueFieldCombinations != null) { - Set
newUniqueFieldCombinations = new HashSet
(); - for (FieldSet fieldCombo : this.uniqueFieldCombinations) { - FieldSet newFieldCombo = new FieldSet(); - for (Integer sourceField : fieldCombo) { - FieldSet targetField = props.getForwardingTargetFields(input, sourceField); - - if (targetField == null || targetField.size() == 0) { - newFieldCombo = null; - break; - } else { - // use any field of target fields for now. We should use something like - // field equivalence sets in the future. - if (targetField.size() > 1) { - LOG.warn( - "Found that a field is forwarded to more than one target field in " - + "semantic forwarded field information. Will only use the field with the lowest index."); - } - newFieldCombo = newFieldCombo.addField(targetField.toArray()[0]); - } - } - if (newFieldCombo != null) { - newUniqueFieldCombinations.add(newFieldCombo); - } - } - if (!newUniqueFieldCombinations.isEmpty()) { - gp.uniqueFieldCombinations = newUniqueFieldCombinations; - } - } - - return gp; - } - - public void parameterizeChannel( - Channel channel, - boolean globalDopChange, - ExecutionMode exchangeMode, - boolean breakPipeline) { - - ShipStrategyType shipType; - FieldList partitionKeys; - boolean[] sortDirection; - Partitioner partitioner; - - switch (this.partitioning) { - case RANDOM_PARTITIONED: - shipType = - globalDopChange - ? ShipStrategyType.PARTITION_RANDOM - : ShipStrategyType.FORWARD; - partitionKeys = null; - sortDirection = null; - partitioner = null; - break; - - case FULL_REPLICATION: - shipType = ShipStrategyType.BROADCAST; - partitionKeys = null; - sortDirection = null; - partitioner = null; - break; - - case ANY_PARTITIONING: - case HASH_PARTITIONED: - shipType = ShipStrategyType.PARTITION_HASH; - partitionKeys = Utils.createOrderedFromSet(this.partitioningFields); - sortDirection = null; - partitioner = null; - break; - - case RANGE_PARTITIONED: - shipType = ShipStrategyType.PARTITION_RANGE; - partitionKeys = this.ordering.getInvolvedIndexes(); - sortDirection = this.ordering.getFieldSortDirections(); - partitioner = null; - break; - - case FORCED_REBALANCED: - shipType = ShipStrategyType.PARTITION_RANDOM; - partitionKeys = null; - sortDirection = null; - partitioner = null; - break; - - case CUSTOM_PARTITIONING: - shipType = ShipStrategyType.PARTITION_CUSTOM; - partitionKeys = this.partitioningFields; - sortDirection = null; - partitioner = this.customPartitioner; - break; - - default: - throw new CompilerException("Unsupported partitioning strategy"); - } - - channel.setDataDistribution(this.distribution); - DataExchangeMode exMode = DataExchangeMode.select(exchangeMode, shipType, breakPipeline); - channel.setShipStrategy(shipType, partitionKeys, sortDirection, partitioner, exMode); - } - - // ------------------------------------------------------------------------ - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((partitioning == null) ? 0 : partitioning.ordinal()); - result = - prime * result + ((partitioningFields == null) ? 0 : partitioningFields.hashCode()); - result = prime * result + ((ordering == null) ? 0 : ordering.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof GlobalProperties) { - final GlobalProperties other = (GlobalProperties) obj; - return (this.partitioning == other.partitioning) - && (this.ordering == other.ordering - || (this.ordering != null && this.ordering.equals(other.ordering))) - && (this.partitioningFields == other.partitioningFields - || (this.partitioningFields != null - && this.partitioningFields.equals(other.partitioningFields))) - && (this.uniqueFieldCombinations == other.uniqueFieldCombinations - || (this.uniqueFieldCombinations != null - && this.uniqueFieldCombinations.equals( - other.uniqueFieldCombinations))); - } else { - return false; - } - } - - @Override - public String toString() { - final StringBuilder bld = - new StringBuilder( - "GlobalProperties [partitioning=" - + partitioning - + (this.partitioningFields == null - ? "" - : ", on fields " + this.partitioningFields) - + (this.ordering == null - ? "" - : ", with ordering " + this.ordering)); - - if (this.uniqueFieldCombinations == null) { - bld.append(']'); - } else { - bld.append(" - Unique field groups: "); - bld.append(this.uniqueFieldCombinations); - bld.append(']'); - } - return bld.toString(); - } - - @Override - public GlobalProperties clone() { - final GlobalProperties newProps = new GlobalProperties(); - newProps.partitioning = this.partitioning; - newProps.partitioningFields = this.partitioningFields; - newProps.ordering = this.ordering; - newProps.distribution = this.distribution; - newProps.customPartitioner = this.customPartitioner; - newProps.uniqueFieldCombinations = - this.uniqueFieldCombinations == null - ? null - : new HashSet
(this.uniqueFieldCombinations); - return newProps; - } - - // -------------------------------------------------------------------------------------------- - - public static GlobalProperties combine(GlobalProperties gp1, GlobalProperties gp2) { - if (gp1.isFullyReplicated()) { - if (gp2.isFullyReplicated()) { - return new GlobalProperties(); - } else { - return gp2; - } - } else if (gp2.isFullyReplicated()) { - return gp1; - } else if (gp1.ordering != null) { - return gp1; - } else if (gp2.ordering != null) { - return gp2; - } else if (gp1.partitioningFields != null) { - return gp1; - } else if (gp2.partitioningFields != null) { - return gp2; - } else if (gp1.uniqueFieldCombinations != null) { - return gp1; - } else if (gp2.uniqueFieldCombinations != null) { - return gp2; - } else if (gp1.getPartitioning().isPartitioned()) { - return gp1; - } else if (gp2.getPartitioning().isPartitioned()) { - return gp2; - } else { - return gp1; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/InterestingProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/InterestingProperties.java deleted file mode 100644 index d84ff74bdf5c8..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/InterestingProperties.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.optimizer.dataproperties; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dag.TwoInputNode; - -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; - -/** - * Interesting properties are propagated from parent operators to child operators. They tell the - * child what data properties would help the parent in operating in a cheaper fashion. A reduce - * operator, for example, tells its child that partitioned data would help. If the child is a join - * operator, it can use that knowledge to favor strategies that leave the data in a partitioned - * form. - * - *

More on optimization with interesting properties can be found in the works on the volcano- and - * cascades optimizer framework. - */ -public class InterestingProperties implements Cloneable { - - private Set - globalProps; // the global properties, i.e. properties across partitions - - private Set - localProps; // the local properties, i.e. properties within partitions - - // ------------------------------------------------------------------------ - - public InterestingProperties() { - this.globalProps = new HashSet(); - this.localProps = new HashSet(); - } - - /** - * Private constructor for cloning purposes. - * - * @param globalProps The global properties for this new object. - * @param localProps The local properties for this new object. - */ - private InterestingProperties( - Set globalProps, Set localProps) { - this.globalProps = globalProps; - this.localProps = localProps; - } - - // ------------------------------------------------------------------------ - - public void addGlobalProperties(RequestedGlobalProperties props) { - this.globalProps.add(props); - } - - public void addLocalProperties(RequestedLocalProperties props) { - this.localProps.add(props); - } - - public void addInterestingProperties(InterestingProperties other) { - this.globalProps.addAll(other.globalProps); - this.localProps.addAll(other.localProps); - } - - /** - * Gets the interesting local properties. - * - * @return The interesting local properties. - */ - public Set getLocalProperties() { - return this.localProps; - } - - /** - * Gets the interesting global properties. - * - * @return The interesting global properties. - */ - public Set getGlobalProperties() { - return this.globalProps; - } - - public InterestingProperties filterByCodeAnnotations(OptimizerNode node, int input) { - InterestingProperties iProps = new InterestingProperties(); - SemanticProperties props; - if (node instanceof SingleInputNode || node instanceof TwoInputNode) { - props = node.getSemanticProperties(); - } else { - props = new SemanticProperties.EmptySemanticProperties(); - } - - for (RequestedGlobalProperties rgp : this.globalProps) { - RequestedGlobalProperties filtered = rgp.filterBySemanticProperties(props, input); - if (filtered != null && !filtered.isTrivial()) { - iProps.addGlobalProperties(filtered); - } - } - for (RequestedLocalProperties rlp : this.localProps) { - RequestedLocalProperties filtered = rlp.filterBySemanticProperties(props, input); - if (filtered != null && !filtered.isTrivial()) { - iProps.addLocalProperties(filtered); - } - } - return iProps; - } - - public void dropTrivials() { - for (Iterator iter = this.globalProps.iterator(); - iter.hasNext(); ) { - RequestedGlobalProperties gp = iter.next(); - if (gp.isTrivial()) { - iter.remove(); - break; - } - } - - for (Iterator iter = this.localProps.iterator(); - iter.hasNext(); ) { - RequestedLocalProperties lp = iter.next(); - if (lp.isTrivial()) { - iter.remove(); - break; - } - } - } - - // ------------------------------------------------------------------------ - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((globalProps == null) ? 0 : globalProps.hashCode()); - result = prime * result + ((localProps == null) ? 0 : localProps.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof InterestingProperties) { - InterestingProperties other = (InterestingProperties) obj; - return this.globalProps.equals(other.globalProps) - && this.localProps.equals(other.localProps); - } else { - return false; - } - } - - @Override - public String toString() { - return "InterestingProperties [globalProps=" - + this.globalProps - + ", localProps=" - + this.localProps - + " ]"; - } - - @Override - public InterestingProperties clone() { - HashSet globalProps = new HashSet(); - for (RequestedGlobalProperties p : this.globalProps) { - globalProps.add(p.clone()); - } - HashSet localProps = new HashSet(); - for (RequestedLocalProperties p : this.localProps) { - localProps.add(p.clone()); - } - - return new InterestingProperties(globalProps, localProps); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/LocalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/LocalProperties.java deleted file mode 100644 index 21144527483dd..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/LocalProperties.java +++ /dev/null @@ -1,323 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashSet; -import java.util.Set; - -/** - * This class represents local properties of the data. A local property is a property that exists - * within the data of a single partition, such as sort order, or data grouping. - */ -public class LocalProperties implements Cloneable { - - public static final Logger LOG = LoggerFactory.getLogger(GlobalProperties.class); - - public static final LocalProperties EMPTY = new LocalProperties(); - - // -------------------------------------------------------------------------------------------- - - private Ordering ordering; // order inside a partition, null if not ordered - - private FieldList groupedFields; // fields by which the stream is grouped. null if not grouped. - - private Set

uniqueFields; // fields whose value combination is unique in the stream - - // -------------------------------------------------------------------------------------------- - - /** Default constructor for trivial local properties. No order, no grouping, no uniqueness. */ - public LocalProperties() {} - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the key order. - * - * @return The key order, or null if nothing is ordered. - */ - public Ordering getOrdering() { - return ordering; - } - - /** - * Gets the grouped fields. - * - * @return The grouped fields, or null if nothing is grouped. - */ - public FieldList getGroupedFields() { - return this.groupedFields; - } - - /** - * Gets the fields whose combination is unique within the data set. - * - * @return The unique field combination, or null if nothing is unique. - */ - public Set
getUniqueFields() { - return this.uniqueFields; - } - - /** - * Checks whether the given set of fields is unique, as specified in these local properties. - * - * @param set The set to check. - * @return True, if the given column combination is unique, false if not. - */ - public boolean areFieldsUnique(FieldSet set) { - return this.uniqueFields != null && this.uniqueFields.contains(set); - } - - /** - * Adds a combination of fields that are unique in these data properties. - * - * @param uniqueFields The fields that are unique in these data properties. - */ - public LocalProperties addUniqueFields(FieldSet uniqueFields) { - LocalProperties copy = clone(); - - if (copy.uniqueFields == null) { - copy.uniqueFields = new HashSet
(); - } - copy.uniqueFields.add(uniqueFields); - return copy; - } - - public LocalProperties clearUniqueFieldSets() { - if (this.uniqueFields == null || this.uniqueFields.isEmpty()) { - return this; - } else { - LocalProperties copy = new LocalProperties(); - copy.ordering = this.ordering; - copy.groupedFields = this.groupedFields; - return copy; - } - } - - /** Checks, if the properties in this object are trivial, i.e. only standard values. */ - public boolean isTrivial() { - return ordering == null && this.groupedFields == null && this.uniqueFields == null; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Filters these LocalProperties by the fields that are forwarded to the output as described by - * the SemanticProperties. - * - * @param props The semantic properties holding information about forwarded fields. - * @param input The index of the input. - * @return The filtered LocalProperties - */ - public LocalProperties filterBySemanticProperties(SemanticProperties props, int input) { - - if (props == null) { - throw new NullPointerException("SemanticProperties may not be null."); - } - - LocalProperties returnProps = new LocalProperties(); - - // check if sorting is preserved - if (this.ordering != null) { - Ordering newOrdering = new Ordering(); - - for (int i = 0; i < this.ordering.getInvolvedIndexes().size(); i++) { - int sourceField = this.ordering.getInvolvedIndexes().get(i); - FieldSet targetField = props.getForwardingTargetFields(input, sourceField); - if (targetField == null || targetField.size() == 0) { - if (i == 0) { - // order fully destroyed - newOrdering = null; - break; - } else { - // order partially preserved - break; - } - } else { - // use any field of target fields for now. We should use something like field - // equivalence sets in the future. - if (targetField.size() > 1) { - LOG.warn( - "Found that a field is forwarded to more than one target field in " - + "semantic forwarded field information. Will only use the field with the lowest index."); - } - newOrdering.appendOrdering( - targetField.toArray()[0], - this.ordering.getType(i), - this.ordering.getOrder(i)); - } - } - - returnProps.ordering = newOrdering; - if (newOrdering != null) { - returnProps.groupedFields = newOrdering.getInvolvedIndexes(); - } else { - returnProps.groupedFields = null; - } - } - // check if grouping is preserved - else if (this.groupedFields != null) { - FieldList newGroupedFields = new FieldList(); - - for (Integer sourceField : this.groupedFields) { - FieldSet targetField = props.getForwardingTargetFields(input, sourceField); - if (targetField == null || targetField.size() == 0) { - newGroupedFields = null; - break; - } else { - // use any field of target fields for now. We should use something like field - // equivalence sets in the future. - if (targetField.size() > 1) { - LOG.warn( - "Found that a field is forwarded to more than one target field in " - + "semantic forwarded field information. Will only use the field with the lowest index."); - } - newGroupedFields = newGroupedFields.addField(targetField.toArray()[0]); - } - } - returnProps.groupedFields = newGroupedFields; - } - - if (this.uniqueFields != null) { - Set
newUniqueFields = new HashSet
(); - for (FieldSet fields : this.uniqueFields) { - FieldSet newFields = new FieldSet(); - for (Integer sourceField : fields) { - FieldSet targetField = props.getForwardingTargetFields(input, sourceField); - - if (targetField == null || targetField.size() == 0) { - newFields = null; - break; - } else { - // use any field of target fields for now. We should use something like - // field equivalence sets in the future. - if (targetField.size() > 1) { - LOG.warn( - "Found that a field is forwarded to more than one target field in " - + "semantic forwarded field information. Will only use the field with the lowest index."); - } - newFields = newFields.addField(targetField.toArray()[0]); - } - } - if (newFields != null) { - newUniqueFields.add(newFields); - } - } - - if (!newUniqueFields.isEmpty()) { - returnProps.uniqueFields = newUniqueFields; - } else { - returnProps.uniqueFields = null; - } - } - - return returnProps; - } - // -------------------------------------------------------------------------------------------- - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (this.ordering == null ? 0 : this.ordering.hashCode()); - result = prime * result + (this.groupedFields == null ? 0 : this.groupedFields.hashCode()); - result = prime * result + (this.uniqueFields == null ? 0 : this.uniqueFields.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof LocalProperties) { - final LocalProperties other = (LocalProperties) obj; - return (ordering == other.getOrdering() - || (ordering != null && ordering.equals(other.getOrdering()))) - && (groupedFields == other.getGroupedFields() - || (groupedFields != null - && groupedFields.equals(other.getGroupedFields()))) - && (uniqueFields == other.getUniqueFields() - || (uniqueFields != null - && uniqueFields.equals(other.getUniqueFields()))); - } else { - return false; - } - } - - @Override - public String toString() { - return "LocalProperties [ordering=" - + this.ordering - + ", grouped=" - + this.groupedFields - + ", unique=" - + this.uniqueFields - + "]"; - } - - @Override - public LocalProperties clone() { - LocalProperties copy = new LocalProperties(); - copy.ordering = this.ordering; - copy.groupedFields = this.groupedFields; - copy.uniqueFields = - (this.uniqueFields == null ? null : new HashSet
(this.uniqueFields)); - return copy; - } - - // -------------------------------------------------------------------------------------------- - - public static LocalProperties combine(LocalProperties lp1, LocalProperties lp2) { - if (lp1.ordering != null) { - return lp1; - } else if (lp2.ordering != null) { - return lp2; - } else if (lp1.groupedFields != null) { - return lp1; - } else if (lp2.groupedFields != null) { - return lp2; - } else if (lp1.uniqueFields != null && !lp1.uniqueFields.isEmpty()) { - return lp1; - } else if (lp2.uniqueFields != null && !lp2.uniqueFields.isEmpty()) { - return lp2; - } else { - return lp1; - } - } - - // -------------------------------------------------------------------------------------------- - - public static LocalProperties forOrdering(Ordering o) { - LocalProperties props = new LocalProperties(); - props.ordering = o; - props.groupedFields = o.getInvolvedIndexes(); - return props; - } - - public static LocalProperties forGrouping(FieldList groupedFields) { - LocalProperties props = new LocalProperties(); - props.groupedFields = groupedFields; - return props; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/PartitioningProperty.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/PartitioningProperty.java deleted file mode 100644 index 553205c7b298c..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/PartitioningProperty.java +++ /dev/null @@ -1,110 +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.optimizer.dataproperties; - -/** - * An enumeration of the different types of distributing data across partitions or parallel workers. - */ -public enum PartitioningProperty { - - /** - * Any possible way of data distribution, including random partitioning and full replication. - */ - ANY_DISTRIBUTION, - - /** - * A random disjunct (non-replicated) data distribution, where each datum is contained in one - * partition only. This is for example the result of parallel scans of data in a file system - * like HDFS, or the result of a round-robin data distribution. - */ - RANDOM_PARTITIONED, - - /** A hash partitioning on a certain key. */ - HASH_PARTITIONED, - - /** A range partitioning on a certain key. */ - RANGE_PARTITIONED, - - /** - * A not further specified partitioning on a key (hash-, or range partitioning, or some other - * scheme even). - */ - ANY_PARTITIONING, - - /** Full replication of the data to each parallel instance. */ - FULL_REPLICATION, - - /** - * A forced even re-balancing. All partitions are guaranteed to have almost the same number of - * records. - */ - FORCED_REBALANCED, - - /** - * A custom partitioning, accompanied by a {@link - * org.apache.flink.api.common.functions.Partitioner}. - */ - CUSTOM_PARTITIONING; - - /** - * Checks, if this property represents in fact a partitioning. That is, whether this property is - * not equal to PartitionProperty.FULL_REPLICATION. - * - * @return True, if this enum constant is unequal to - * PartitionProperty.FULL_REPLICATION, false otherwise. - */ - public boolean isPartitioned() { - return this != FULL_REPLICATION && this != FORCED_REBALANCED && this != ANY_DISTRIBUTION; - } - - /** - * Checks, if this property represents a full replication. - * - * @return True, if this enum constant is equal to PartitionProperty.FULL_REPLICATION, - * false otherwise. - */ - public boolean isReplication() { - return this == FULL_REPLICATION; - } - - /** - * Checks if this property presents a partitioning that is not random, but on a partitioning - * key. - * - * @return True, if the data is partitioned on a key. - */ - public boolean isPartitionedOnKey() { - return isPartitioned() && this != RANDOM_PARTITIONED; - } - - /** - * Checks, if this property represents a partitioning that is computable. A computable - * partitioning can be recreated through an algorithm. If two sets of data are to be - * co-partitioned, it is crucial, that the partitioning schemes are computable. - * - *

Examples for computable partitioning schemes are hash- or range-partitioning. An example - * for a non-computable partitioning is the implicit partitioning that exists though a globally - * unique key. - * - * @return True, if this enum constant is a re-computable partitioning. - */ - public boolean isComputablyPartitioned() { - return this == HASH_PARTITIONED || this == RANGE_PARTITIONED || this == CUSTOM_PARTITIONING; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java deleted file mode 100644 index 5eb14215eedf6..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalProperties.java +++ /dev/null @@ -1,490 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -/** - * This class represents the global properties of the data that are requested by an operator. - * Operators request the global properties they need for correct execution. This list is an example - * of global properties requested by certain operators: - * - *

    - *
  • "groupBy/reduce" will request the data to be partitioned in some way after the key fields. - *
  • "map" will request the data to be in an arbitrary distribution - it has no prerequisites - *
  • "join" will request certain properties for each input. This class represents the properties - * on an input alone. The properties may be partitioning on the key fields, or a combination - * of replication on one input and anything-but-replication on the other input. - *
- */ -public final class RequestedGlobalProperties implements Cloneable { - - private PartitioningProperty partitioning; // the type partitioning - - private FieldSet partitioningFields; // the fields which are partitioned - - private Ordering ordering; // order of the partitioned fields, if it is an ordered (range) range - // partitioning - - private DataDistribution - dataDistribution; // optional data distribution, for a range partitioning - - private Partitioner customPartitioner; // optional, partitioner for custom partitioning - - // -------------------------------------------------------------------------------------------- - - /** Initializes the global properties with no partitioning. */ - public RequestedGlobalProperties() { - this.partitioning = PartitioningProperty.RANDOM_PARTITIONED; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Sets these properties to request a hash partitioning on the given fields. - * - *

If the fields are provided as {@link FieldSet}, then any permutation of the fields is a - * valid partitioning, including subsets. If the fields are given as a {@link FieldList}, then - * only an exact partitioning on the fields matches this requested partitioning. - * - * @param partitionedFields The key fields for the partitioning. - */ - public void setHashPartitioned(FieldSet partitionedFields) { - if (partitionedFields == null) { - throw new NullPointerException(); - } - this.partitioning = PartitioningProperty.HASH_PARTITIONED; - this.partitioningFields = partitionedFields; - this.ordering = null; - } - - public void setRangePartitioned(Ordering ordering) { - this.setRangePartitioned(ordering, null); - } - - public void setRangePartitioned(Ordering ordering, DataDistribution dataDistribution) { - if (ordering == null) { - throw new NullPointerException(); - } - this.partitioning = PartitioningProperty.RANGE_PARTITIONED; - this.ordering = ordering; - this.partitioningFields = null; - this.dataDistribution = dataDistribution; - } - - /** - * Sets these properties to request some partitioning on the given fields. This will allow both - * hash partitioning and range partitioning to match. - * - *

If the fields are provided as {@link FieldSet}, then any permutation of the fields is a - * valid partitioning, including subsets. If the fields are given as a {@link FieldList}, then - * only an exact partitioning on the fields matches this requested partitioning. - * - * @param partitionedFields The key fields for the partitioning. - */ - public void setAnyPartitioning(FieldSet partitionedFields) { - if (partitionedFields == null) { - throw new NullPointerException(); - } - this.partitioning = PartitioningProperty.ANY_PARTITIONING; - this.partitioningFields = partitionedFields; - this.ordering = null; - } - - public void setRandomPartitioning() { - this.partitioning = PartitioningProperty.RANDOM_PARTITIONED; - this.partitioningFields = null; - this.ordering = null; - } - - public void setAnyDistribution() { - this.partitioning = PartitioningProperty.ANY_DISTRIBUTION; - this.partitioningFields = null; - this.ordering = null; - } - - public void setFullyReplicated() { - this.partitioning = PartitioningProperty.FULL_REPLICATION; - this.partitioningFields = null; - this.ordering = null; - } - - public void setForceRebalancing() { - this.partitioning = PartitioningProperty.FORCED_REBALANCED; - this.partitioningFields = null; - this.ordering = null; - } - - /** - * Sets these properties to request a custom partitioning with the given {@link Partitioner} - * instance. - * - *

If the fields are provided as {@link FieldSet}, then any permutation of the fields is a - * valid partitioning, including subsets. If the fields are given as a {@link FieldList}, then - * only an exact partitioning on the fields matches this requested partitioning. - * - * @param partitionedFields The key fields for the partitioning. - */ - public void setCustomPartitioned(FieldSet partitionedFields, Partitioner partitioner) { - if (partitionedFields == null || partitioner == null) { - throw new NullPointerException(); - } - - this.partitioning = PartitioningProperty.CUSTOM_PARTITIONING; - this.partitioningFields = partitionedFields; - this.ordering = null; - this.customPartitioner = partitioner; - } - - /** - * Gets the partitioning property. - * - * @return The partitioning property. - */ - public PartitioningProperty getPartitioning() { - return partitioning; - } - - /** - * Gets the fields on which the data is partitioned. - * - * @return The partitioning fields. - */ - public FieldSet getPartitionedFields() { - return this.partitioningFields; - } - - /** - * Gets the key order. - * - * @return The key order. - */ - public Ordering getOrdering() { - return this.ordering; - } - - /** - * Gets the data distribution. - * - * @return The data distribution. - */ - public DataDistribution getDataDistribution() { - return this.dataDistribution; - } - - /** - * Gets the custom partitioner associated with these properties. - * - * @return The custom partitioner associated with these properties. - */ - public Partitioner getCustomPartitioner() { - return customPartitioner; - } - - /** Checks, if the properties in this object are trivial, i.e. only standard values. */ - public boolean isTrivial() { - return this.partitioning == null - || this.partitioning == PartitioningProperty.RANDOM_PARTITIONED; - } - - /** This method resets the properties to a state where no properties are given. */ - public void reset() { - this.partitioning = PartitioningProperty.RANDOM_PARTITIONED; - this.ordering = null; - this.partitioningFields = null; - this.dataDistribution = null; - this.customPartitioner = null; - } - - /** - * Filters these properties by what can be preserved by the given SemanticProperties when - * propagated down to the given input. - * - * @param props The SemanticProperties which define which fields are preserved. - * @param input The index of the operator's input. - * @return The filtered RequestedGlobalProperties - */ - public RequestedGlobalProperties filterBySemanticProperties( - SemanticProperties props, int input) { - // no semantic properties available. All global properties are filtered. - if (props == null) { - throw new NullPointerException("SemanticProperties may not be null."); - } - - RequestedGlobalProperties rgProp = new RequestedGlobalProperties(); - - switch (this.partitioning) { - case FULL_REPLICATION: - case FORCED_REBALANCED: - case CUSTOM_PARTITIONING: - case RANDOM_PARTITIONED: - case ANY_DISTRIBUTION: - // make sure that certain properties are not pushed down - return null; - case HASH_PARTITIONED: - case ANY_PARTITIONING: - FieldSet newFields; - if (this.partitioningFields instanceof FieldList) { - newFields = new FieldList(); - } else { - newFields = new FieldSet(); - } - - for (Integer targetField : this.partitioningFields) { - int sourceField = props.getForwardingSourceField(input, targetField); - if (sourceField >= 0) { - newFields = newFields.addField(sourceField); - } else { - // partial partitionings are not preserved to avoid skewed partitioning - return null; - } - } - rgProp.partitioning = this.partitioning; - rgProp.partitioningFields = newFields; - return rgProp; - case RANGE_PARTITIONED: - // range partitioning - Ordering newOrdering = new Ordering(); - for (int i = 0; i < this.ordering.getInvolvedIndexes().size(); i++) { - int value = this.ordering.getInvolvedIndexes().get(i); - int sourceField = props.getForwardingSourceField(input, value); - if (sourceField >= 0) { - newOrdering.appendOrdering( - sourceField, this.ordering.getType(i), this.ordering.getOrder(i)); - } else { - return null; - } - } - rgProp.partitioning = this.partitioning; - rgProp.ordering = newOrdering; - rgProp.dataDistribution = this.dataDistribution; - return rgProp; - default: - throw new RuntimeException("Unknown partitioning type encountered."); - } - } - - /** - * Checks, if this set of interesting properties, is met by the given produced properties. - * - * @param props The properties for which to check whether they meet these properties. - * @return True, if the properties are met, false otherwise. - */ - public boolean isMetBy(GlobalProperties props) { - if (this.partitioning == PartitioningProperty.ANY_DISTRIBUTION) { - return true; - } else if (this.partitioning == PartitioningProperty.FULL_REPLICATION) { - return props.isFullyReplicated(); - } else if (props.isFullyReplicated()) { - return false; - } else if (this.partitioning == PartitioningProperty.RANDOM_PARTITIONED) { - return true; - } else if (this.partitioning == PartitioningProperty.ANY_PARTITIONING) { - return checkCompatiblePartitioningFields(props); - } else if (this.partitioning == PartitioningProperty.HASH_PARTITIONED) { - return props.getPartitioning() == PartitioningProperty.HASH_PARTITIONED - && checkCompatiblePartitioningFields(props); - } else if (this.partitioning == PartitioningProperty.RANGE_PARTITIONED) { - return props.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED - && props.matchesOrderedPartitioning(this.ordering); - } else if (this.partitioning == PartitioningProperty.FORCED_REBALANCED) { - return props.getPartitioning() == PartitioningProperty.FORCED_REBALANCED; - } else if (this.partitioning == PartitioningProperty.CUSTOM_PARTITIONING) { - return props.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING - && checkCompatiblePartitioningFields(props) - && props.getCustomPartitioner().equals(this.customPartitioner); - - } else { - throw new CompilerException("Properties matching logic leaves open cases."); - } - } - - /** - * Parametrizes the ship strategy fields of a channel such that the channel produces the desired - * global properties. - * - * @param channel The channel to parametrize. - * @param globalDopChange Flag indicating whether the parallelism changes between sender and - * receiver. - * @param exchangeMode The mode of data exchange (pipelined, always batch, batch only on - * shuffle, ...) - * @param breakPipeline Indicates whether this data exchange should break pipelines (unless - * pipelines are forced). - */ - public void parameterizeChannel( - Channel channel, - boolean globalDopChange, - ExecutionMode exchangeMode, - boolean breakPipeline) { - - // safety check. Fully replicated input must be preserved. - if (channel.getSource().getGlobalProperties().isFullyReplicated() - && !(this.partitioning == PartitioningProperty.FULL_REPLICATION - || this.partitioning == PartitioningProperty.ANY_DISTRIBUTION)) { - throw new CompilerException( - "Fully replicated input must be preserved " - + "and may not be converted into another global property."); - } - - // if we request nothing, then we need no special strategy. forward, if the number of - // instances remains - // the same, randomly repartition otherwise - if (isTrivial() || this.partitioning == PartitioningProperty.ANY_DISTRIBUTION) { - ShipStrategyType shipStrategy = - globalDopChange ? ShipStrategyType.PARTITION_RANDOM : ShipStrategyType.FORWARD; - - DataExchangeMode em = - DataExchangeMode.select(exchangeMode, shipStrategy, breakPipeline); - channel.setShipStrategy(shipStrategy, em); - return; - } - - final GlobalProperties inGlobals = channel.getSource().getGlobalProperties(); - // if we have no global parallelism change, check if we have already compatible global - // properties - if (!globalDopChange && isMetBy(inGlobals)) { - DataExchangeMode em = - DataExchangeMode.select(exchangeMode, ShipStrategyType.FORWARD, breakPipeline); - channel.setShipStrategy(ShipStrategyType.FORWARD, em); - return; - } - - // if we fall through the conditions until here, we need to re-establish - ShipStrategyType shipType; - FieldList partitionKeys; - boolean[] sortDirection; - Partitioner partitioner; - - switch (this.partitioning) { - case FULL_REPLICATION: - shipType = ShipStrategyType.BROADCAST; - partitionKeys = null; - sortDirection = null; - partitioner = null; - break; - - case ANY_PARTITIONING: - case HASH_PARTITIONED: - shipType = ShipStrategyType.PARTITION_HASH; - partitionKeys = Utils.createOrderedFromSet(this.partitioningFields); - sortDirection = null; - partitioner = null; - break; - - case RANGE_PARTITIONED: - shipType = ShipStrategyType.PARTITION_RANGE; - partitionKeys = this.ordering.getInvolvedIndexes(); - sortDirection = this.ordering.getFieldSortDirections(); - partitioner = null; - - if (this.dataDistribution != null) { - channel.setDataDistribution(this.dataDistribution); - } - break; - - case FORCED_REBALANCED: - shipType = ShipStrategyType.PARTITION_FORCED_REBALANCE; - partitionKeys = null; - sortDirection = null; - partitioner = null; - break; - - case CUSTOM_PARTITIONING: - shipType = ShipStrategyType.PARTITION_CUSTOM; - partitionKeys = Utils.createOrderedFromSet(this.partitioningFields); - sortDirection = null; - partitioner = this.customPartitioner; - break; - - default: - throw new CompilerException( - "Invalid partitioning to create through a data exchange: " - + this.partitioning.name()); - } - - DataExchangeMode exMode = DataExchangeMode.select(exchangeMode, shipType, breakPipeline); - channel.setShipStrategy(shipType, partitionKeys, sortDirection, partitioner, exMode); - } - - // ------------------------------------------------------------------------ - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((partitioning == null) ? 0 : partitioning.ordinal()); - result = - prime * result + ((partitioningFields == null) ? 0 : partitioningFields.hashCode()); - result = prime * result + ((ordering == null) ? 0 : ordering.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof RequestedGlobalProperties) { - RequestedGlobalProperties other = (RequestedGlobalProperties) obj; - return (ordering == other.getOrdering() - || (ordering != null && ordering.equals(other.getOrdering()))) - && (partitioning == other.getPartitioning()) - && (partitioningFields == other.partitioningFields - || (partitioningFields != null - && partitioningFields.equals(other.getPartitionedFields()))); - } else { - return false; - } - } - - @Override - public String toString() { - return "Requested Global Properties [partitioning=" - + partitioning - + (this.partitioningFields == null ? "" : ", on fields " + this.partitioningFields) - + (this.ordering == null ? "" : ", with ordering " + this.ordering) - + "]"; - } - - public RequestedGlobalProperties clone() { - try { - return (RequestedGlobalProperties) super.clone(); - } catch (CloneNotSupportedException cnse) { - // should never happen, but propagate just in case - throw new RuntimeException(cnse); - } - } - - private boolean checkCompatiblePartitioningFields(GlobalProperties props) { - if (this.partitioningFields instanceof FieldList) { - // partitioningFields as FieldList requires strict checking! - return props.isExactlyPartitionedOnFields((FieldList) this.partitioningFields); - } else { - return props.isPartitionedOnFields(this.partitioningFields); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedLocalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedLocalProperties.java deleted file mode 100644 index 7f7ae26959742..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/RequestedLocalProperties.java +++ /dev/null @@ -1,270 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import java.util.Arrays; - -/** - * This class represents the local properties of the data that are requested by an operator. Local - * properties are the properties within one partition. Operators request the local properties they - * need for correct execution. Here are some example local properties requested by certain - * operators: - * - *

    - *
  • "groupBy/reduce" will request the data to be grouped on the key fields. - *
  • A sort-merge join will request the data from each input to be sorted on the respective join - * key. - *
- */ -public class RequestedLocalProperties implements Cloneable { - - private Ordering ordering; // order inside a partition, null if not ordered - - private FieldSet groupedFields; // fields by which the stream is grouped. null if not grouped. - - // -------------------------------------------------------------------------------------------- - - /** Default constructor for trivial local properties. No order, no grouping, no uniqueness. */ - public RequestedLocalProperties() {} - - /** - * Creates interesting properties for the given ordering. - * - * @param ordering The interesting ordering. - */ - public RequestedLocalProperties(Ordering ordering) { - this.ordering = ordering; - } - - /** - * Creates interesting properties for the given grouping. - * - * @param groupedFields The set of fields whose grouping is interesting. - */ - public RequestedLocalProperties(FieldSet groupedFields) { - this.groupedFields = groupedFields; - } - - /** - * This constructor is used only for internal copy creation. - * - * @param ordering The ordering represented by these local properties. - * @param groupedFields The grouped fields for these local properties. - */ - private RequestedLocalProperties(Ordering ordering, FieldSet groupedFields) { - this.ordering = ordering; - this.groupedFields = groupedFields; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Gets the key order. - * - * @return The key order, or null if nothing is ordered. - */ - public Ordering getOrdering() { - return ordering; - } - - /** - * Sets the order for these interesting local properties. - * - * @param ordering The order to set. - */ - public void setOrdering(Ordering ordering) { - this.ordering = ordering; - } - - /** - * Gets the grouped fields. - * - * @return The grouped fields, or null if nothing is grouped. - */ - public FieldSet getGroupedFields() { - return this.groupedFields; - } - - /** - * Sets the fields that are grouped in these data properties. - * - * @param groupedFields The fields that are grouped in these data properties. - */ - public void setGroupedFields(FieldSet groupedFields) { - this.groupedFields = groupedFields; - } - - /** Checks, if the properties in this object are trivial, i.e. only standard values. */ - public boolean isTrivial() { - return ordering == null && this.groupedFields == null; - } - - /** This method resets the local properties to a state where no properties are given. */ - public void reset() { - this.ordering = null; - this.groupedFields = null; - } - - // -------------------------------------------------------------------------------------------- - - /** - * Filters these properties by what can be preserved by the given SemanticProperties when - * propagated down to the given input. - * - * @param props The SemanticProperties which define which fields are preserved. - * @param input The index of the operator's input. - * @return The filtered RequestedLocalProperties - */ - public RequestedLocalProperties filterBySemanticProperties( - SemanticProperties props, int input) { - - // no semantic properties, all local properties are filtered - if (props == null) { - throw new NullPointerException("SemanticProperties may not be null."); - } - - if (this.ordering != null) { - Ordering newOrdering = new Ordering(); - - for (int i = 0; i < this.ordering.getInvolvedIndexes().size(); i++) { - int targetField = this.ordering.getInvolvedIndexes().get(i); - int sourceField = props.getForwardingSourceField(input, targetField); - if (sourceField >= 0) { - newOrdering.appendOrdering( - sourceField, this.ordering.getType(i), this.ordering.getOrder(i)); - } else { - return null; - } - } - return new RequestedLocalProperties(newOrdering); - - } else if (this.groupedFields != null) { - FieldSet newGrouping = new FieldSet(); - - // check, whether the local key grouping is preserved - for (Integer targetField : this.groupedFields) { - int sourceField = props.getForwardingSourceField(input, targetField); - if (sourceField >= 0) { - newGrouping = newGrouping.addField(sourceField); - } else { - return null; - } - } - return new RequestedLocalProperties(newGrouping); - } else { - return null; - } - } - - /** - * Checks, if this set of properties, as interesting properties, is met by the given properties. - * - * @param other The properties for which to check whether they meet these properties. - * @return True, if the properties are met, false otherwise. - */ - public boolean isMetBy(LocalProperties other) { - if (this.ordering != null) { - // we demand an ordering - return other.getOrdering() != null && this.ordering.isMetBy(other.getOrdering()); - } else if (this.groupedFields != null) { - // check if the other fields are unique - if (other.getGroupedFields() != null - && other.getGroupedFields().isValidUnorderedPrefix(this.groupedFields)) { - return true; - } else { - return other.areFieldsUnique(this.groupedFields); - } - } else { - return true; - } - } - - /** - * Parametrizes the local strategy fields of a channel such that the channel produces the - * desired local properties. - * - * @param channel The channel to parametrize. - */ - public void parameterizeChannel(Channel channel) { - LocalProperties current = channel.getLocalProperties(); - - if (isMetBy(current)) { - // we are met, all is good - channel.setLocalStrategy(LocalStrategy.NONE); - } else if (this.ordering != null) { - channel.setLocalStrategy( - LocalStrategy.SORT, - this.ordering.getInvolvedIndexes(), - this.ordering.getFieldSortDirections()); - } else if (this.groupedFields != null) { - boolean[] dirs = new boolean[this.groupedFields.size()]; - Arrays.fill(dirs, true); - channel.setLocalStrategy( - LocalStrategy.SORT, Utils.createOrderedFromSet(this.groupedFields), dirs); - } else { - channel.setLocalStrategy(LocalStrategy.NONE); - } - } - - // -------------------------------------------------------------------------------------------- - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (this.ordering == null ? 0 : this.ordering.hashCode()); - result = prime * result + (this.groupedFields == null ? 0 : this.groupedFields.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof RequestedLocalProperties) { - final RequestedLocalProperties other = (RequestedLocalProperties) obj; - return (this.ordering == other.ordering - || (this.ordering != null && this.ordering.equals(other.ordering))) - && (this.groupedFields == other.groupedFields - || (this.groupedFields != null - && this.groupedFields.equals(other.groupedFields))); - } else { - return false; - } - } - - @Override - public String toString() { - return "Requested Local Properties [ordering=" - + this.ordering - + ", grouped=" - + this.groupedFields - + "]"; - } - - @Override - public RequestedLocalProperties clone() { - return new RequestedLocalProperties(this.ordering, this.groupedFields); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractJoinDescriptor.java deleted file mode 100644 index 5c08175c38547..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractJoinDescriptor.java +++ /dev/null @@ -1,153 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** Defines the possible global properties for a join. */ -public abstract class AbstractJoinDescriptor extends OperatorDescriptorDual { - - private final boolean broadcastFirstAllowed; - private final boolean broadcastSecondAllowed; - private final boolean repartitionAllowed; - - private Partitioner customPartitioner; - - protected AbstractJoinDescriptor(FieldList keys1, FieldList keys2) { - this(keys1, keys2, true, true, true); - } - - protected AbstractJoinDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2); - - this.broadcastFirstAllowed = broadcastFirstAllowed; - this.broadcastSecondAllowed = broadcastSecondAllowed; - this.repartitionAllowed = repartitionAllowed; - } - - public void setCustomPartitioner(Partitioner partitioner) { - customPartitioner = partitioner; - } - - @Override - protected List createPossibleGlobalProperties() { - ArrayList pairs = new ArrayList(); - - if (repartitionAllowed) { - // partition both (hash or custom) - if (this.customPartitioner == null) { - - // we accept compatible partitionings of any type - RequestedGlobalProperties partitioned_left_any = new RequestedGlobalProperties(); - RequestedGlobalProperties partitioned_right_any = new RequestedGlobalProperties(); - partitioned_left_any.setAnyPartitioning(this.keys1); - partitioned_right_any.setAnyPartitioning(this.keys2); - pairs.add(new GlobalPropertiesPair(partitioned_left_any, partitioned_right_any)); - - // add strict hash partitioning of both inputs on their full key sets - RequestedGlobalProperties partitioned_left_hash = new RequestedGlobalProperties(); - RequestedGlobalProperties partitioned_right_hash = new RequestedGlobalProperties(); - partitioned_left_hash.setHashPartitioned(this.keys1); - partitioned_right_hash.setHashPartitioned(this.keys2); - pairs.add(new GlobalPropertiesPair(partitioned_left_hash, partitioned_right_hash)); - } else { - RequestedGlobalProperties partitioned_left = new RequestedGlobalProperties(); - partitioned_left.setCustomPartitioned(this.keys1, this.customPartitioner); - - RequestedGlobalProperties partitioned_right = new RequestedGlobalProperties(); - partitioned_right.setCustomPartitioned(this.keys2, this.customPartitioner); - - return Collections.singletonList( - new GlobalPropertiesPair(partitioned_left, partitioned_right)); - } - - RequestedGlobalProperties partitioned1 = new RequestedGlobalProperties(); - if (customPartitioner == null) { - partitioned1.setAnyPartitioning(this.keys1); - } else { - partitioned1.setCustomPartitioned(this.keys1, this.customPartitioner); - } - - RequestedGlobalProperties partitioned2 = new RequestedGlobalProperties(); - if (customPartitioner == null) { - partitioned2.setAnyPartitioning(this.keys2); - } else { - partitioned2.setCustomPartitioned(this.keys2, this.customPartitioner); - } - - pairs.add(new GlobalPropertiesPair(partitioned1, partitioned2)); - } - - if (broadcastSecondAllowed) { - // replicate second - RequestedGlobalProperties any1 = new RequestedGlobalProperties(); - RequestedGlobalProperties replicated2 = new RequestedGlobalProperties(); - replicated2.setFullyReplicated(); - pairs.add(new GlobalPropertiesPair(any1, replicated2)); - } - - if (broadcastFirstAllowed) { - // replicate first - RequestedGlobalProperties replicated1 = new RequestedGlobalProperties(); - replicated1.setFullyReplicated(); - RequestedGlobalProperties any2 = new RequestedGlobalProperties(); - pairs.add(new GlobalPropertiesPair(replicated1, any2)); - } - return pairs; - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - if (requested1.getPartitioning().isPartitionedOnKey() - && requested2.getPartitioning().isPartitionedOnKey()) { - return checkCompatibilityBasedOnDiversePartitioningProperty(produced1, produced2); - } else { - return true; - } - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties in1, GlobalProperties in2) { - GlobalProperties gp = GlobalProperties.combine(in1, in2); - if (gp.getUniqueFieldCombination() != null - && gp.getUniqueFieldCombination().size() > 0 - && gp.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gp.setAnyPartitioning(gp.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gp.clearUniqueFieldCombinations(); - return gp; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractOperatorDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractOperatorDescriptor.java deleted file mode 100644 index 9b14552b5df6e..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractOperatorDescriptor.java +++ /dev/null @@ -1,32 +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.optimizer.operators; - -import org.apache.flink.runtime.operators.DriverStrategy; - -/** The root of all descriptions of an operator to the optimizer. */ -public interface AbstractOperatorDescriptor { - - /** - * Gets the runtime strategy to be executed by this operator. - * - * @return The runtime strategy. - */ - DriverStrategy getStrategy(); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java deleted file mode 100644 index 81331af93b13c..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractSortMergeJoinDescriptor.java +++ /dev/null @@ -1,92 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.util.Utils; - -import java.util.Collections; -import java.util.List; - -public abstract class AbstractSortMergeJoinDescriptor extends AbstractJoinDescriptor { - - public AbstractSortMergeJoinDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2); - } - - public AbstractSortMergeJoinDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties sort1 = - new RequestedLocalProperties(Utils.createOrdering(this.keys1)); - RequestedLocalProperties sort2 = - new RequestedLocalProperties(Utils.createOrdering(this.keys2)); - return Collections.singletonList(new LocalPropertiesPair(sort1, sort2)); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - int numRelevantFields = this.keys1.size(); - return checkSameOrdering(produced1, produced2, numRelevantFields); - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - boolean[] inputOrders = in1.getLocalProperties().getOrdering().getFieldSortDirections(); - - if (inputOrders == null || inputOrders.length < this.keys1.size()) { - throw new CompilerException( - "BUG: The input strategy does not sufficiently describe the sort orders for a merge operator."); - } else if (inputOrders.length > this.keys1.size()) { - boolean[] tmp = new boolean[this.keys1.size()]; - System.arraycopy(inputOrders, 0, tmp, 0, tmp.length); - inputOrders = tmp; - } - - String nodeName = String.format("%s (%s)", getNodeName(), node.getOperator().getName()); - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2, inputOrders); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - LocalProperties comb = LocalProperties.combine(in1, in2); - return comb.clearUniqueFieldSets(); - } - - protected abstract String getNodeName(); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupCombineProperties.java deleted file mode 100644 index 6175545e38984..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupCombineProperties.java +++ /dev/null @@ -1,76 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public final class AllGroupCombineProperties extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.ALL_GROUP_COMBINE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, - "GroupCombine (" + node.getOperator().getName() + ")", - in, - DriverStrategy.ALL_GROUP_COMBINE); - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupReduceProperties.java deleted file mode 100644 index 4df3c96c1c063..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupReduceProperties.java +++ /dev/null @@ -1,76 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public final class AllGroupReduceProperties extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.ALL_GROUP_REDUCE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, - "GroupReduce (" + node.getOperator().getName() + ")", - in, - DriverStrategy.ALL_GROUP_REDUCE); - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java deleted file mode 100644 index 756f73d484d60..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllGroupWithPartialPreGroupProperties.java +++ /dev/null @@ -1,118 +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.optimizer.operators; - -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.GroupReduceNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import java.util.Collections; -import java.util.List; - -public final class AllGroupWithPartialPreGroupProperties extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.ALL_GROUP_REDUCE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - if (in.getShipStrategy() == ShipStrategyType.FORWARD) { - // locally connected, directly instantiate - return new SingleInputPlanNode( - node, - "GroupReduce (" + node.getOperator().getName() + ")", - in, - DriverStrategy.ALL_GROUP_REDUCE); - } else { - // non forward case.plug in a combiner - Channel toCombiner = new Channel(in.getSource()); - toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - - // create an input node for combine with same parallelism as input node - GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode(); - combinerNode.setParallelism(in.getSource().getParallelism()); - - SingleInputPlanNode combiner = - new SingleInputPlanNode( - combinerNode, - "Combine (" + node.getOperator().getName() + ")", - toCombiner, - DriverStrategy.ALL_GROUP_REDUCE_COMBINE); - combiner.setCosts(new Costs(0, 0)); - combiner.initProperties( - toCombiner.getGlobalProperties(), toCombiner.getLocalProperties()); - - Channel toReducer = new Channel(combiner); - toReducer.setShipStrategy( - in.getShipStrategy(), - in.getShipStrategyKeys(), - in.getShipStrategySortOrder(), - in.getDataExchangeMode()); - - toReducer.setLocalStrategy( - in.getLocalStrategy(), - in.getLocalStrategyKeys(), - in.getLocalStrategySortOrder()); - return new SingleInputPlanNode( - node, - "GroupReduce (" + node.getOperator().getName() + ")", - toReducer, - DriverStrategy.ALL_GROUP_REDUCE); - } - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java deleted file mode 100644 index 68851e796a31d..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AllReduceProperties.java +++ /dev/null @@ -1,110 +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.optimizer.operators; - -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.ReduceNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import java.util.Collections; -import java.util.List; - -public final class AllReduceProperties extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.ALL_REDUCE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - if (in.getShipStrategy() == ShipStrategyType.FORWARD) { - // locally connected, directly instantiate - return new SingleInputPlanNode( - node, - "Reduce (" + node.getOperator().getName() + ")", - in, - DriverStrategy.ALL_REDUCE); - } else { - // non forward case.plug in a combiner - Channel toCombiner = new Channel(in.getSource()); - toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - - // create an input node for combine with same parallelism as input node - ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode(); - combinerNode.setParallelism(in.getSource().getParallelism()); - - SingleInputPlanNode combiner = - new SingleInputPlanNode( - combinerNode, - "Combine (" + node.getOperator().getName() + ")", - toCombiner, - DriverStrategy.ALL_REDUCE); - combiner.setCosts(new Costs(0, 0)); - combiner.initProperties( - toCombiner.getGlobalProperties(), toCombiner.getLocalProperties()); - - Channel toReducer = new Channel(combiner); - toReducer.setShipStrategy( - in.getShipStrategy(), - in.getShipStrategyKeys(), - in.getShipStrategySortOrder(), - in.getDataExchangeMode()); - toReducer.setLocalStrategy( - in.getLocalStrategy(), - in.getLocalStrategyKeys(), - in.getLocalStrategySortOrder()); - - return new SingleInputPlanNode( - node, - "Reduce (" + node.getOperator().getName() + ")", - toReducer, - DriverStrategy.ALL_REDUCE); - } - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - return new GlobalProperties(); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/BinaryUnionOpDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/BinaryUnionOpDescriptor.java deleted file mode 100644 index 87a669996a655..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/BinaryUnionOpDescriptor.java +++ /dev/null @@ -1,122 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.BinaryUnionNode; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.BinaryUnionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** */ -public class BinaryUnionOpDescriptor extends OperatorDescriptorDual { - - public BinaryUnionOpDescriptor() { - super(); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.UNION; - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.emptyList(); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.emptyList(); - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - return new BinaryUnionPlanNode((BinaryUnionNode) node, in1, in2); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties in1, GlobalProperties in2) { - GlobalProperties newProps = new GlobalProperties(); - - if (in1.getPartitioning() == PartitioningProperty.HASH_PARTITIONED - && in2.getPartitioning() == PartitioningProperty.HASH_PARTITIONED - && in1.getPartitioningFields().equals(in2.getPartitioningFields())) { - newProps.setHashPartitioned(in1.getPartitioningFields()); - } else if (in1.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED - && in2.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED - && in1.getPartitioningOrdering().equals(in2.getPartitioningOrdering()) - && (in1.getDataDistribution() == null && in2.getDataDistribution() == null - || in1.getDataDistribution() != null - && in1.getDataDistribution().equals(in2.getDataDistribution()))) { - if (in1.getDataDistribution() == null) { - newProps.setRangePartitioned(in1.getPartitioningOrdering()); - } else { - newProps.setRangePartitioned( - in1.getPartitioningOrdering(), in1.getDataDistribution()); - } - } else if (in1.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING - && in2.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING - && in1.getPartitioningFields().equals(in2.getPartitioningFields()) - && in1.getCustomPartitioner().equals(in2.getCustomPartitioner())) { - newProps.setCustomPartitioned(in1.getPartitioningFields(), in1.getCustomPartitioner()); - } else if (in1.getPartitioning() == PartitioningProperty.FORCED_REBALANCED - && in2.getPartitioning() == PartitioningProperty.FORCED_REBALANCED) { - newProps.setForcedRebalanced(); - } else if (in1.getPartitioning() == PartitioningProperty.FULL_REPLICATION - && in2.getPartitioning() == PartitioningProperty.FULL_REPLICATION) { - newProps.setFullyReplicated(); - } - - return newProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - // all local properties are destroyed - return new LocalProperties(); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - return true; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CartesianProductDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CartesianProductDescriptor.java deleted file mode 100644 index 7fc7cbb5fa1cd..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CartesianProductDescriptor.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.optimizer.operators; - -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -public abstract class CartesianProductDescriptor extends OperatorDescriptorDual { - - private final boolean allowBroadcastFirst; - private final boolean allowBroadcastSecond; - - protected CartesianProductDescriptor( - boolean allowBroadcastFirst, boolean allowBroadcastSecond) { - if (!(allowBroadcastFirst || allowBroadcastSecond)) { - throw new IllegalArgumentException(); - } - - this.allowBroadcastFirst = allowBroadcastFirst; - this.allowBroadcastSecond = allowBroadcastSecond; - } - - @Override - protected List createPossibleGlobalProperties() { - ArrayList pairs = new ArrayList(); - - if (this.allowBroadcastFirst) { - // replicate first - RequestedGlobalProperties replicated1 = new RequestedGlobalProperties(); - replicated1.setFullyReplicated(); - RequestedGlobalProperties any2 = new RequestedGlobalProperties(); - pairs.add(new GlobalPropertiesPair(replicated1, any2)); - } - - if (this.allowBroadcastSecond) { - // replicate second - RequestedGlobalProperties any1 = new RequestedGlobalProperties(); - RequestedGlobalProperties replicated2 = new RequestedGlobalProperties(); - replicated2.setFullyReplicated(); - pairs.add(new GlobalPropertiesPair(any1, replicated2)); - } - - return pairs; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - return true; - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - return new DualInputPlanNode( - node, "Cross (" + node.getOperator().getName() + ")", in1, in2, getStrategy()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties in1, GlobalProperties in2) { - GlobalProperties gp = GlobalProperties.combine(in1, in2); - if (gp.getUniqueFieldCombination() != null - && gp.getUniqueFieldCombination().size() > 0 - && gp.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gp.setAnyPartitioning(gp.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gp.clearUniqueFieldCombinations(); - return gp; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java deleted file mode 100644 index abebf93a46979..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java +++ /dev/null @@ -1,207 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class CoGroupDescriptor extends OperatorDescriptorDual { - - private final Ordering ordering1; // ordering on the first input - private final Ordering ordering2; // ordering on the second input - - private Partitioner customPartitioner; - - public CoGroupDescriptor(FieldList keys1, FieldList keys2) { - this(keys1, keys2, null, null); - } - - public CoGroupDescriptor( - FieldList keys1, - FieldList keys2, - Ordering additionalOrdering1, - Ordering additionalOrdering2) { - super(keys1, keys2); - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - if (additionalOrdering1 != null) { - this.ordering1 = new Ordering(); - for (Integer key : this.keys1) { - this.ordering1.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - for (int i = 0; i < additionalOrdering1.getNumberOfFields(); i++) { - Integer field = additionalOrdering1.getFieldNumber(i); - Order order = additionalOrdering1.getOrder(i); - this.ordering1.appendOrdering(field, additionalOrdering1.getType(i), order); - } - } else { - this.ordering1 = Utils.createOrdering(this.keys1); - } - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - if (additionalOrdering2 != null) { - this.ordering2 = new Ordering(); - for (Integer key : this.keys2) { - this.ordering2.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - for (int i = 0; i < additionalOrdering2.getNumberOfFields(); i++) { - Integer field = additionalOrdering2.getFieldNumber(i); - Order order = additionalOrdering2.getOrder(i); - this.ordering2.appendOrdering(field, additionalOrdering2.getType(i), order); - } - } else { - this.ordering2 = Utils.createOrdering(this.keys2); - } - } - - public void setCustomPartitioner(Partitioner customPartitioner) { - this.customPartitioner = customPartitioner; - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.CO_GROUP; - } - - @Override - protected List createPossibleGlobalProperties() { - - if (this.customPartitioner == null) { - - // we accept compatible partitionings of any type - RequestedGlobalProperties partitioned_left_any = new RequestedGlobalProperties(); - RequestedGlobalProperties partitioned_right_any = new RequestedGlobalProperties(); - partitioned_left_any.setAnyPartitioning(this.keys1); - partitioned_right_any.setAnyPartitioning(this.keys2); - - // add strict hash partitioning of both inputs on their full key sets - RequestedGlobalProperties partitioned_left_hash = new RequestedGlobalProperties(); - RequestedGlobalProperties partitioned_right_hash = new RequestedGlobalProperties(); - partitioned_left_hash.setHashPartitioned(this.keys1); - partitioned_right_hash.setHashPartitioned(this.keys2); - - return Arrays.asList( - new GlobalPropertiesPair(partitioned_left_any, partitioned_right_any), - new GlobalPropertiesPair(partitioned_left_hash, partitioned_right_hash)); - } else { - RequestedGlobalProperties partitioned_left = new RequestedGlobalProperties(); - partitioned_left.setCustomPartitioned(this.keys1, this.customPartitioner); - - RequestedGlobalProperties partitioned_right = new RequestedGlobalProperties(); - partitioned_right.setCustomPartitioned(this.keys2, this.customPartitioner); - - return Collections.singletonList( - new GlobalPropertiesPair(partitioned_left, partitioned_right)); - } - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties sort1 = new RequestedLocalProperties(this.ordering1); - RequestedLocalProperties sort2 = new RequestedLocalProperties(this.ordering2); - return Collections.singletonList(new LocalPropertiesPair(sort1, sort2)); - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - return checkCompatibilityBasedOnDiversePartitioningProperty(produced1, produced2); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - int numRelevantFields = this.keys1.size(); - return checkSameOrdering(produced1, produced2, numRelevantFields); - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - boolean[] inputOrders = - in1.getLocalProperties().getOrdering() == null - ? null - : in1.getLocalProperties().getOrdering().getFieldSortDirections(); - - if (inputOrders == null || inputOrders.length < this.keys1.size()) { - throw new CompilerException( - "BUG: The input strategy does not sufficiently describe the sort orders for a CoGroup operator."); - } else if (inputOrders.length > this.keys1.size()) { - boolean[] tmp = new boolean[this.keys1.size()]; - System.arraycopy(inputOrders, 0, tmp, 0, tmp.length); - inputOrders = tmp; - } - - return new DualInputPlanNode( - node, - "CoGroup (" + node.getOperator().getName() + ")", - in1, - in2, - DriverStrategy.CO_GROUP, - this.keys1, - this.keys2, - inputOrders); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties in1, GlobalProperties in2) { - GlobalProperties gp = GlobalProperties.combine(in1, in2); - if (gp.getUniqueFieldCombination() != null - && gp.getUniqueFieldCombination().size() > 0 - && gp.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gp.setAnyPartitioning(gp.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gp.clearUniqueFieldCombinations(); - return gp; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - LocalProperties comb = LocalProperties.combine(in1, in2); - return comb.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupRawDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupRawDescriptor.java deleted file mode 100644 index df1f6d339dc95..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupRawDescriptor.java +++ /dev/null @@ -1,200 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** */ -public class CoGroupRawDescriptor extends OperatorDescriptorDual { - - private final Ordering ordering1; // ordering on the first input - private final Ordering ordering2; // ordering on the second input - - public CoGroupRawDescriptor(FieldList keys1, FieldList keys2) { - this(keys1, keys2, null, null); - } - - public CoGroupRawDescriptor( - FieldList keys1, - FieldList keys2, - Ordering additionalOrdering1, - Ordering additionalOrdering2) { - super(keys1, keys2); - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - if (additionalOrdering1 != null) { - this.ordering1 = new Ordering(); - for (Integer key : this.keys1) { - this.ordering1.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - for (int i = 0; i < additionalOrdering1.getNumberOfFields(); i++) { - Integer field = additionalOrdering1.getFieldNumber(i); - Order order = additionalOrdering1.getOrder(i); - this.ordering1.appendOrdering(field, additionalOrdering1.getType(i), order); - } - } else { - this.ordering1 = Utils.createOrdering(this.keys1); - } - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - if (additionalOrdering2 != null) { - this.ordering2 = new Ordering(); - for (Integer key : this.keys2) { - this.ordering2.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - for (int i = 0; i < additionalOrdering2.getNumberOfFields(); i++) { - Integer field = additionalOrdering2.getFieldNumber(i); - Order order = additionalOrdering2.getOrder(i); - this.ordering2.appendOrdering(field, additionalOrdering2.getType(i), order); - } - } else { - this.ordering2 = Utils.createOrdering(this.keys2); - } - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.CO_GROUP_RAW; - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties partitioned1 = new RequestedGlobalProperties(); - partitioned1.setHashPartitioned(this.keys1); - RequestedGlobalProperties partitioned2 = new RequestedGlobalProperties(); - partitioned2.setHashPartitioned(this.keys2); - return Collections.singletonList( - new OperatorDescriptorDual.GlobalPropertiesPair(partitioned1, partitioned2)); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties sort1 = new RequestedLocalProperties(this.ordering1); - RequestedLocalProperties sort2 = new RequestedLocalProperties(this.ordering2); - return Collections.singletonList( - new OperatorDescriptorDual.LocalPropertiesPair(sort1, sort2)); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - int numRelevantFields = this.keys1.size(); - - Ordering prod1 = produced1.getOrdering(); - Ordering prod2 = produced2.getOrdering(); - - if (prod1 == null - || prod2 == null - || prod1.getNumberOfFields() < numRelevantFields - || prod2.getNumberOfFields() < numRelevantFields) { - throw new CompilerException( - "The given properties do not meet this operators requirements."); - } - - for (int i = 0; i < numRelevantFields; i++) { - if (prod1.getOrder(i) != prod2.getOrder(i)) { - return false; - } - } - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - boolean[] inputOrders = - in1.getLocalProperties().getOrdering() == null - ? null - : in1.getLocalProperties().getOrdering().getFieldSortDirections(); - - if (inputOrders == null || inputOrders.length < this.keys1.size()) { - throw new CompilerException( - "BUG: The input strategy does not sufficiently describe the sort orders for a CoGroup operator."); - } else if (inputOrders.length > this.keys1.size()) { - boolean[] tmp = new boolean[this.keys1.size()]; - System.arraycopy(inputOrders, 0, tmp, 0, tmp.length); - inputOrders = tmp; - } - - return new DualInputPlanNode( - node, - "CoGroup (" + node.getOperator().getName() + ")", - in1, - in2, - DriverStrategy.CO_GROUP_RAW, - this.keys1, - this.keys2, - inputOrders); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties in1, GlobalProperties in2) { - GlobalProperties gp = GlobalProperties.combine(in1, in2); - if (gp.getUniqueFieldCombination() != null - && gp.getUniqueFieldCombination().size() > 0 - && gp.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gp.setAnyPartitioning(gp.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gp.clearUniqueFieldCombinations(); - return gp; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - LocalProperties comb = LocalProperties.combine(in1, in2); - return comb.clearUniqueFieldSets(); - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - return produced1.getPartitioning() == produced2.getPartitioning() - && (produced1.getCustomPartitioner() == null - ? produced2.getCustomPartitioner() == null - : produced1 - .getCustomPartitioner() - .equals(produced2.getCustomPartitioner())); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupWithSolutionSetFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupWithSolutionSetFirstDescriptor.java deleted file mode 100644 index 1aa53860d63ee..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupWithSolutionSetFirstDescriptor.java +++ /dev/null @@ -1,89 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** */ -public class CoGroupWithSolutionSetFirstDescriptor extends CoGroupDescriptor { - - public CoGroupWithSolutionSetFirstDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties none = new RequestedLocalProperties(); - RequestedLocalProperties sort = - new RequestedLocalProperties(Utils.createOrdering(this.keys2)); - return Collections.singletonList(new LocalPropertiesPair(none, sort)); - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - boolean[] inputOrders = - in2.getLocalProperties().getOrdering() == null - ? null - : in2.getLocalProperties().getOrdering().getFieldSortDirections(); - - if (inputOrders == null || inputOrders.length < this.keys2.size()) { - throw new CompilerException( - "BUG: The input strategy does not sufficiently describe the sort orders for a CoGroup operator."); - } else if (inputOrders.length > this.keys2.size()) { - boolean[] tmp = new boolean[this.keys2.size()]; - System.arraycopy(inputOrders, 0, tmp, 0, tmp.length); - inputOrders = tmp; - } - - return new DualInputPlanNode( - node, - "CoGroup (" + node.getOperator().getName() + ")", - in1, - in2, - DriverStrategy.CO_GROUP, - this.keys1, - this.keys2, - inputOrders); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return in2; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupWithSolutionSetSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupWithSolutionSetSecondDescriptor.java deleted file mode 100644 index 48ab89f172857..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupWithSolutionSetSecondDescriptor.java +++ /dev/null @@ -1,57 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.util.Utils; - -import java.util.Collections; -import java.util.List; - -/** */ -public class CoGroupWithSolutionSetSecondDescriptor extends CoGroupDescriptor { - - public CoGroupWithSolutionSetSecondDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties sort = - new RequestedLocalProperties(Utils.createOrdering(this.keys1)); - RequestedLocalProperties none = new RequestedLocalProperties(); - return Collections.singletonList(new LocalPropertiesPair(sort, none)); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return in1; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossBlockOuterFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossBlockOuterFirstDescriptor.java deleted file mode 100644 index c6531c71f8044..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossBlockOuterFirstDescriptor.java +++ /dev/null @@ -1,44 +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.optimizer.operators; - -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class CrossBlockOuterFirstDescriptor extends CartesianProductDescriptor { - - public CrossBlockOuterFirstDescriptor() { - this(true, true); - } - - public CrossBlockOuterFirstDescriptor( - boolean allowBroadcastFirst, boolean allowBroadcastSecond) { - super(allowBroadcastFirst, allowBroadcastSecond); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_FIRST; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossBlockOuterSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossBlockOuterSecondDescriptor.java deleted file mode 100644 index 32416246b5cc1..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossBlockOuterSecondDescriptor.java +++ /dev/null @@ -1,44 +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.optimizer.operators; - -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class CrossBlockOuterSecondDescriptor extends CartesianProductDescriptor { - - public CrossBlockOuterSecondDescriptor() { - this(true, true); - } - - public CrossBlockOuterSecondDescriptor( - boolean allowBroadcastFirst, boolean allowBroadcastSecond) { - super(allowBroadcastFirst, allowBroadcastSecond); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.NESTEDLOOP_BLOCKED_OUTER_SECOND; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossStreamOuterFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossStreamOuterFirstDescriptor.java deleted file mode 100644 index 246451c1d3ecf..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossStreamOuterFirstDescriptor.java +++ /dev/null @@ -1,52 +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.optimizer.operators; - -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class CrossStreamOuterFirstDescriptor extends CartesianProductDescriptor { - - public CrossStreamOuterFirstDescriptor() { - this(true, true); - } - - public CrossStreamOuterFirstDescriptor( - boolean allowBroadcastFirst, boolean allowBroadcastSecond) { - super(allowBroadcastFirst, allowBroadcastSecond); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.NESTEDLOOP_STREAMED_OUTER_FIRST; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - // uniqueness becomes grouping with streamed nested loops - if ((in1.getGroupedFields() == null || in1.getGroupedFields().size() == 0) - && in1.getUniqueFields() != null - && in1.getUniqueFields().size() > 0) { - return LocalProperties.forGrouping( - in1.getUniqueFields().iterator().next().toFieldList()); - } else { - return in1.clearUniqueFieldSets(); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossStreamOuterSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossStreamOuterSecondDescriptor.java deleted file mode 100644 index 4ad9786737dcd..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CrossStreamOuterSecondDescriptor.java +++ /dev/null @@ -1,52 +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.optimizer.operators; - -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class CrossStreamOuterSecondDescriptor extends CartesianProductDescriptor { - - public CrossStreamOuterSecondDescriptor() { - this(true, true); - } - - public CrossStreamOuterSecondDescriptor( - boolean allowBroadcastFirst, boolean allowBroadcastSecond) { - super(allowBroadcastFirst, allowBroadcastSecond); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.NESTEDLOOP_STREAMED_OUTER_SECOND; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - // uniqueness becomes grouping with streamed nested loops - if ((in2.getGroupedFields() == null || in2.getGroupedFields().size() == 0) - && in2.getUniqueFields() != null - && in2.getUniqueFields().size() > 0) { - return LocalProperties.forGrouping( - in2.getUniqueFields().iterator().next().toFieldList()); - } else { - return in2.clearUniqueFieldSets(); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/FilterDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/FilterDescriptor.java deleted file mode 100644 index ae443c9f9d3c0..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/FilterDescriptor.java +++ /dev/null @@ -1,67 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class FilterDescriptor extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.FLAT_MAP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, "Filter (" + node.getOperator().getName() + ")", in, DriverStrategy.FLAT_MAP); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setAnyDistribution(); - return Collections.singletonList(rgp); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/FlatMapDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/FlatMapDescriptor.java deleted file mode 100644 index f37b21af4c1f9..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/FlatMapDescriptor.java +++ /dev/null @@ -1,78 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class FlatMapDescriptor extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.FLAT_MAP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, - "FlatMap (" + node.getOperator().getName() + ")", - in, - DriverStrategy.FLAT_MAP); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setAnyDistribution(); - return Collections.singletonList(rgp); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java deleted file mode 100644 index 33073823e93c3..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupCombineProperties.java +++ /dev/null @@ -1,124 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** - * The properties file belonging to the GroupCombineNode. It translates the GroupCombine operation - * to the driver strategy SORTED_GROUP_COMBINE and sets the relevant grouping and sorting keys. - * - * @see org.apache.flink.optimizer.dag.GroupCombineNode - */ -public final class GroupCombineProperties extends OperatorDescriptorSingle { - - private final Ordering - ordering; // ordering that we need to use if an additional ordering is requested - - public GroupCombineProperties(FieldSet groupKeys, Ordering additionalOrderKeys) { - super(groupKeys); - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - - this.ordering = new Ordering(); - for (Integer key : this.keyList) { - this.ordering.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - if (additionalOrderKeys != null) { - for (int i = 0; i < additionalOrderKeys.getNumberOfFields(); i++) { - Integer field = additionalOrderKeys.getFieldNumber(i); - Order order = additionalOrderKeys.getOrder(i); - this.ordering.appendOrdering(field, additionalOrderKeys.getType(i), order); - } - } - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.SORTED_GROUP_COMBINE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - node.setParallelism(in.getSource().getParallelism()); - - // sorting key info - SingleInputPlanNode singleInputPlanNode = - new SingleInputPlanNode( - node, - "GroupCombine (" + node.getOperator().getName() + ")", - in, // reuse the combine strategy also used in the group reduce - DriverStrategy.SORTED_GROUP_COMBINE, - this.keyList); - - // set sorting comparator key info - singleInputPlanNode.setDriverKeyInfo( - this.ordering.getInvolvedIndexes(), this.ordering.getFieldSortDirections(), 0); - // set grouping comparator key info - singleInputPlanNode.setDriverKeyInfo(this.keyList, 1); - - return singleInputPlanNode; - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties props = new RequestedGlobalProperties(); - props.setRandomPartitioning(); - return Collections.singletonList(props); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceProperties.java deleted file mode 100644 index ed8743b406062..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceProperties.java +++ /dev/null @@ -1,136 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public final class GroupReduceProperties extends OperatorDescriptorSingle { - - private final Ordering - ordering; // ordering that we need to use if an additional ordering is requested - - private final Partitioner customPartitioner; - - public GroupReduceProperties(FieldSet keys) { - this(keys, null, null); - } - - public GroupReduceProperties(FieldSet keys, Ordering additionalOrderKeys) { - this(keys, additionalOrderKeys, null); - } - - public GroupReduceProperties(FieldSet keys, Partitioner customPartitioner) { - this(keys, null, customPartitioner); - } - - public GroupReduceProperties( - FieldSet groupKeys, Ordering additionalOrderKeys, Partitioner customPartitioner) { - super(groupKeys); - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - if (additionalOrderKeys != null) { - this.ordering = new Ordering(); - for (Integer key : this.keyList) { - this.ordering.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - for (int i = 0; i < additionalOrderKeys.getNumberOfFields(); i++) { - Integer field = additionalOrderKeys.getFieldNumber(i); - Order order = additionalOrderKeys.getOrder(i); - this.ordering.appendOrdering(field, additionalOrderKeys.getType(i), order); - } - } else { - this.ordering = null; - } - - this.customPartitioner = customPartitioner; - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.SORTED_GROUP_REDUCE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, - "GroupReduce (" + node.getOperator().getName() + ")", - in, - DriverStrategy.SORTED_GROUP_REDUCE, - this.keyList); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties props = new RequestedGlobalProperties(); - - if (customPartitioner == null) { - props.setAnyPartitioning(this.keys); - } else { - props.setCustomPartitioned(this.keys, this.customPartitioner); - } - return Collections.singletonList(props); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties props = new RequestedLocalProperties(); - if (this.ordering == null) { - props.setGroupedFields(this.keys); - } else { - props.setOrdering(this.ordering); - } - return Collections.singletonList(props); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java deleted file mode 100644 index 0b5c2880d6977..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java +++ /dev/null @@ -1,206 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.GroupReduceNode; -import org.apache.flink.optimizer.dag.PartitionNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.List; - -public final class GroupReduceWithCombineProperties extends OperatorDescriptorSingle { - private static final Logger LOG = - LoggerFactory.getLogger(GroupReduceWithCombineProperties.class); - - private final Ordering - ordering; // ordering that we need to use if an additional ordering is requested - - private final Partitioner customPartitioner; - - public GroupReduceWithCombineProperties(FieldSet groupKeys) { - this(groupKeys, null, null); - } - - public GroupReduceWithCombineProperties(FieldSet groupKeys, Ordering additionalOrderKeys) { - this(groupKeys, additionalOrderKeys, null); - } - - public GroupReduceWithCombineProperties(FieldSet groupKeys, Partitioner customPartitioner) { - this(groupKeys, null, customPartitioner); - } - - public GroupReduceWithCombineProperties( - FieldSet groupKeys, Ordering additionalOrderKeys, Partitioner customPartitioner) { - super(groupKeys); - - // if we have an additional ordering, construct the ordering to have primarily the grouping - // fields - if (additionalOrderKeys != null) { - this.ordering = new Ordering(); - for (Integer key : this.keyList) { - this.ordering.appendOrdering(key, null, Order.ANY); - } - - // and next the additional order fields - for (int i = 0; i < additionalOrderKeys.getNumberOfFields(); i++) { - Integer field = additionalOrderKeys.getFieldNumber(i); - Order order = additionalOrderKeys.getOrder(i); - this.ordering.appendOrdering(field, additionalOrderKeys.getType(i), order); - } - } else { - this.ordering = null; - } - - this.customPartitioner = customPartitioner; - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.SORTED_GROUP_REDUCE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - if (in.getShipStrategy() == ShipStrategyType.FORWARD) { - if (in.getSource().getOptimizerNode() instanceof PartitionNode) { - LOG.warn( - "Cannot automatically inject combiner for GroupReduceFunction. Please add an explicit combiner with combineGroup() in front of the partition operator."); - } - // adjust a sort (changes grouping, so it must be for this driver to combining sort - if (in.getLocalStrategy() == LocalStrategy.SORT) { - if (!in.getLocalStrategyKeys().isValidUnorderedPrefix(this.keys)) { - throw new RuntimeException("Bug: Inconsistent sort for group strategy."); - } - in.setLocalStrategy( - LocalStrategy.COMBININGSORT, - in.getLocalStrategyKeys(), - in.getLocalStrategySortOrder()); - } - return new SingleInputPlanNode( - node, - "Reduce (" + node.getOperator().getName() + ")", - in, - DriverStrategy.SORTED_GROUP_REDUCE, - this.keyList); - } else { - // non forward case. all local properties are killed anyways, so we can safely plug in a - // combiner - Channel toCombiner = new Channel(in.getSource()); - toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - - // create an input node for combine with same parallelism as input node - GroupReduceNode combinerNode = ((GroupReduceNode) node).getCombinerUtilityNode(); - combinerNode.setParallelism(in.getSource().getParallelism()); - - SingleInputPlanNode combiner = - new SingleInputPlanNode( - combinerNode, - "Combine (" + node.getOperator().getName() + ")", - toCombiner, - DriverStrategy.SORTED_GROUP_COMBINE); - combiner.setCosts(new Costs(0, 0)); - combiner.initProperties( - toCombiner.getGlobalProperties(), toCombiner.getLocalProperties()); - // set sorting comparator key info - combiner.setDriverKeyInfo(in.getLocalStrategyKeys(), in.getLocalStrategySortOrder(), 0); - // set grouping comparator key info - combiner.setDriverKeyInfo(this.keyList, 1); - - Channel toReducer = new Channel(combiner); - toReducer.setShipStrategy( - in.getShipStrategy(), - in.getShipStrategyKeys(), - in.getShipStrategySortOrder(), - in.getDataExchangeMode()); - if (in.getShipStrategy() == ShipStrategyType.PARTITION_RANGE) { - toReducer.setDataDistribution(in.getDataDistribution()); - } - toReducer.setLocalStrategy( - LocalStrategy.COMBININGSORT, - in.getLocalStrategyKeys(), - in.getLocalStrategySortOrder()); - - return new SingleInputPlanNode( - node, - "Reduce (" + node.getOperator().getName() + ")", - toReducer, - DriverStrategy.SORTED_GROUP_REDUCE, - this.keyList); - } - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties props = new RequestedGlobalProperties(); - if (customPartitioner == null) { - props.setAnyPartitioning(this.keys); - } else { - props.setCustomPartitioned(this.keys, this.customPartitioner); - } - return Collections.singletonList(props); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties props = new RequestedLocalProperties(); - if (this.ordering == null) { - props.setGroupedFields(this.keys); - } else { - props.setOrdering(this.ordering); - } - return Collections.singletonList(props); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildFirstDescriptor.java deleted file mode 100644 index 3c2bcc8e5476f..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildFirstDescriptor.java +++ /dev/null @@ -1,71 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class HashFullOuterJoinBuildFirstDescriptor extends AbstractJoinDescriptor { - - public HashFullOuterJoinBuildFirstDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2, false, false, true); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.FULL_OUTER_HYBRIDHASH_BUILD_FIRST; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - - String nodeName = "FullOuterJoin (" + node.getOperator().getName() + ")"; - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildSecondDescriptor.java deleted file mode 100644 index e49f784ed0b9c..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildSecondDescriptor.java +++ /dev/null @@ -1,70 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class HashFullOuterJoinBuildSecondDescriptor extends AbstractJoinDescriptor { - public HashFullOuterJoinBuildSecondDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2, false, false, true); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.FULL_OUTER_HYBRIDHASH_BUILD_SECOND; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - - String nodeName = "FullOuterJoin (" + node.getOperator().getName() + ")"; - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashJoinBuildFirstProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashJoinBuildFirstProperties.java deleted file mode 100644 index 1a12a8bdf32cb..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashJoinBuildFirstProperties.java +++ /dev/null @@ -1,100 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** */ -public class HashJoinBuildFirstProperties extends AbstractJoinDescriptor { - - public HashJoinBuildFirstProperties(FieldList keys1, FieldList keys2) { - super(keys1, keys2); - } - - public HashJoinBuildFirstProperties( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.HYBRIDHASH_BUILD_FIRST; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - DriverStrategy strategy; - - if (!in1.isOnDynamicPath() && in2.isOnDynamicPath()) { - // sanity check that the first input is cached and remove that cache - if (!in1.getTempMode().isCached()) { - throw new CompilerException( - "No cache at point where static and dynamic parts meet."); - } - in1.setTempMode(in1.getTempMode().makeNonCached()); - strategy = DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED; - } else { - strategy = DriverStrategy.HYBRIDHASH_BUILD_FIRST; - } - return new DualInputPlanNode( - node, - "Join (" + node.getOperator().getName() + ")", - in1, - in2, - strategy, - this.keys1, - this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashJoinBuildSecondProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashJoinBuildSecondProperties.java deleted file mode 100644 index e50cc5ebe1fab..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashJoinBuildSecondProperties.java +++ /dev/null @@ -1,100 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public final class HashJoinBuildSecondProperties extends AbstractJoinDescriptor { - - public HashJoinBuildSecondProperties(FieldList keys1, FieldList keys2) { - super(keys1, keys2); - } - - public HashJoinBuildSecondProperties( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.HYBRIDHASH_BUILD_SECOND; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - DriverStrategy strategy; - - if (!in2.isOnDynamicPath() && in1.isOnDynamicPath()) { - // sanity check that the first input is cached and remove that cache - if (!in2.getTempMode().isCached()) { - throw new CompilerException( - "No cache at point where static and dynamic parts meet."); - } - - in2.setTempMode(in2.getTempMode().makeNonCached()); - strategy = DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED; - } else { - strategy = DriverStrategy.HYBRIDHASH_BUILD_SECOND; - } - return new DualInputPlanNode( - node, - "Join (" + node.getOperator().getName() + ")", - in1, - in2, - strategy, - this.keys1, - this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildFirstDescriptor.java deleted file mode 100644 index ee69b40a58d1b..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildFirstDescriptor.java +++ /dev/null @@ -1,74 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class HashLeftOuterJoinBuildFirstDescriptor extends AbstractJoinDescriptor { - public HashLeftOuterJoinBuildFirstDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, false, broadcastSecondAllowed, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.LEFT_HYBRIDHASH_BUILD_FIRST; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new OperatorDescriptorDual.LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - - String nodeName = "LeftOuterJoin (" + node.getOperator().getName() + ")"; - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java deleted file mode 100644 index e42883e9d5b74..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildSecondDescriptor.java +++ /dev/null @@ -1,76 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class HashLeftOuterJoinBuildSecondDescriptor extends AbstractJoinDescriptor { - - public HashLeftOuterJoinBuildSecondDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, false, broadcastSecondAllowed, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.LEFT_HYBRIDHASH_BUILD_SECOND; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - - String nodeName = "LeftOuterJoin (" + node.getOperator().getName() + ")"; - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java deleted file mode 100644 index 6bc6a7720dbdc..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildFirstDescriptor.java +++ /dev/null @@ -1,76 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class HashRightOuterJoinBuildFirstDescriptor extends AbstractJoinDescriptor { - - public HashRightOuterJoinBuildFirstDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, broadcastFirstAllowed, false, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.RIGHT_HYBRIDHASH_BUILD_FIRST; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - - String nodeName = "RightOuterJoin (" + node.getOperator().getName() + ")"; - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildSecondDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildSecondDescriptor.java deleted file mode 100644 index 1e994dec5eabc..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildSecondDescriptor.java +++ /dev/null @@ -1,74 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class HashRightOuterJoinBuildSecondDescriptor extends AbstractJoinDescriptor { - public HashRightOuterJoinBuildSecondDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, broadcastFirstAllowed, false, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.RIGHT_HYBRIDHASH_BUILD_SECOND; - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - - String nodeName = "RightOuterJoin (" + node.getOperator().getName() + ")"; - return new DualInputPlanNode( - node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/MapDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/MapDescriptor.java deleted file mode 100644 index 015265c457b27..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/MapDescriptor.java +++ /dev/null @@ -1,67 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class MapDescriptor extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.MAP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, "Map (" + node.getOperator().getName() + ")", in, DriverStrategy.MAP); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setAnyDistribution(); - return Collections.singletonList(rgp); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/MapPartitionDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/MapPartitionDescriptor.java deleted file mode 100644 index fe873d4be6dc2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/MapPartitionDescriptor.java +++ /dev/null @@ -1,70 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class MapPartitionDescriptor extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.MAP_PARTITION; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode( - node, - "MapPartition (" + node.getOperator().getName() + ")", - in, - DriverStrategy.MAP_PARTITION); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setAnyDistribution(); - return Collections.singletonList(rgp); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/NoOpDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/NoOpDescriptor.java deleted file mode 100644 index 41e89e62ef3a4..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/NoOpDescriptor.java +++ /dev/null @@ -1,64 +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.optimizer.operators; - -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public class NoOpDescriptor extends OperatorDescriptorSingle { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.UNARY_NO_OP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode(node, "Pipe", in, DriverStrategy.UNARY_NO_OP); - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java deleted file mode 100644 index 8954b5e16b68f..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorDual.java +++ /dev/null @@ -1,314 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; - -import java.util.List; - -/** */ -public abstract class OperatorDescriptorDual implements AbstractOperatorDescriptor { - - protected final FieldList keys1; - protected final FieldList keys2; - - private List globalProps; - private List localProps; - - protected OperatorDescriptorDual() { - this(null, null); - } - - protected OperatorDescriptorDual(FieldList keys1, FieldList keys2) { - this.keys1 = keys1; - this.keys2 = keys2; - } - - public List getPossibleGlobalProperties() { - if (this.globalProps == null) { - this.globalProps = createPossibleGlobalProperties(); - } - - return this.globalProps; - } - - public List getPossibleLocalProperties() { - if (this.localProps == null) { - this.localProps = createPossibleLocalProperties(); - } - - return this.localProps; - } - - protected abstract List createPossibleGlobalProperties(); - - protected abstract List createPossibleLocalProperties(); - - public abstract boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2); - - public abstract boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2); - - public abstract DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node); - - public abstract GlobalProperties computeGlobalProperties( - GlobalProperties in1, GlobalProperties in2); - - public abstract LocalProperties computeLocalProperties( - LocalProperties in1, LocalProperties in2); - - protected boolean checkCompatibilityBasedOnDiversePartitioningProperty( - GlobalProperties produced1, GlobalProperties produced2) { - if (produced1.getPartitioning() == PartitioningProperty.HASH_PARTITIONED - && produced2.getPartitioning() == PartitioningProperty.HASH_PARTITIONED) { - - // both are hash partitioned, check that partitioning fields are equivalently chosen - return checkEquivalentFieldPositionsInKeyFields( - produced1.getPartitioningFields(), produced2.getPartitioningFields()); - - } else if (produced1.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED - && produced2.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED - && produced1.getDataDistribution() != null - && produced2.getDataDistribution() != null) { - - return produced1.getPartitioningFields().size() - == produced2.getPartitioningFields().size() - && checkSameOrdering( - produced1, produced2, produced1.getPartitioningFields().size()) - && produced1.getDataDistribution().equals(produced2.getDataDistribution()); - - } else if (produced1.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING - && produced2.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING) { - - // both use a custom partitioner. Check that both keys are exactly as specified and - // that both the same partitioner - return produced1.getPartitioningFields().isExactMatch(this.keys1) - && produced2.getPartitioningFields().isExactMatch(this.keys2) - && produced1.getCustomPartitioner() != null - && produced2.getCustomPartitioner() != null - && produced1.getCustomPartitioner().equals(produced2.getCustomPartitioner()); - - } else { - - // no other partitioning valid, incl. ANY_PARTITIONING. - // For joins we must ensure that both sides are exactly identically partitioned, - // ANY is not good enough. - return false; - } - } - - protected boolean checkEquivalentFieldPositionsInKeyFields( - FieldList fields1, FieldList fields2) { - - // check number of produced partitioning fields - if (fields1.size() != fields2.size()) { - return false; - } else { - return checkEquivalentFieldPositionsInKeyFields(fields1, fields2, fields1.size()); - } - } - - protected boolean checkEquivalentFieldPositionsInKeyFields( - FieldList fields1, FieldList fields2, int numRelevantFields) { - - // check number of produced partitioning fields - if (fields1.size() < numRelevantFields || fields2.size() < numRelevantFields) { - return false; - } else { - for (int i = 0; i < numRelevantFields; i++) { - int pField1 = fields1.get(i); - int pField2 = fields2.get(i); - // check if position of both produced fields is the same in both requested fields - int j; - for (j = 0; j < this.keys1.size(); j++) { - if (this.keys1.get(j) == pField1 && this.keys2.get(j) == pField2) { - break; - } else if (this.keys1.get(j) != pField1 && this.keys2.get(j) != pField2) { - // do nothing - } else { - return false; - } - } - if (j == this.keys1.size()) { - throw new CompilerException("Fields were not found in key fields."); - } - } - } - return true; - } - - protected boolean checkSameOrdering( - GlobalProperties produced1, GlobalProperties produced2, int numRelevantFields) { - Ordering prod1 = produced1.getPartitioningOrdering(); - Ordering prod2 = produced2.getPartitioningOrdering(); - - if (prod1 == null || prod2 == null) { - throw new CompilerException( - "The given properties do not meet this operators requirements."); - } - - // check that order of fields is equivalent - if (!checkEquivalentFieldPositionsInKeyFields( - prod1.getInvolvedIndexes(), prod2.getInvolvedIndexes(), numRelevantFields)) { - return false; - } - - // check that both inputs have the same directions of order - for (int i = 0; i < numRelevantFields; i++) { - if (prod1.getOrder(i) != prod2.getOrder(i)) { - return false; - } - } - return true; - } - - protected boolean checkSameOrdering( - LocalProperties produced1, LocalProperties produced2, int numRelevantFields) { - Ordering prod1 = produced1.getOrdering(); - Ordering prod2 = produced2.getOrdering(); - - if (prod1 == null || prod2 == null) { - throw new CompilerException( - "The given properties do not meet this operators requirements."); - } - - // check that order of fields is equivalent - if (!checkEquivalentFieldPositionsInKeyFields( - prod1.getInvolvedIndexes(), prod2.getInvolvedIndexes(), numRelevantFields)) { - return false; - } - - // check that both inputs have the same directions of order - for (int i = 0; i < numRelevantFields; i++) { - if (prod1.getOrder(i) != prod2.getOrder(i)) { - return false; - } - } - return true; - } - - // -------------------------------------------------------------------------------------------- - - public static final class GlobalPropertiesPair { - - private final RequestedGlobalProperties props1, props2; - - public GlobalPropertiesPair( - RequestedGlobalProperties props1, RequestedGlobalProperties props2) { - this.props1 = props1; - this.props2 = props2; - } - - public RequestedGlobalProperties getProperties1() { - return this.props1; - } - - public RequestedGlobalProperties getProperties2() { - return this.props2; - } - - @Override - public int hashCode() { - return (this.props1 == null ? 0 : this.props1.hashCode()) - ^ (this.props2 == null ? 0 : this.props2.hashCode()); - } - - @Override - public boolean equals(Object obj) { - if (obj.getClass() == GlobalPropertiesPair.class) { - final GlobalPropertiesPair other = (GlobalPropertiesPair) obj; - - return (this.props1 == null - ? other.props1 == null - : this.props1.equals(other.props1)) - && (this.props2 == null - ? other.props2 == null - : this.props2.equals(other.props2)); - } - return false; - } - - @Override - public String toString() { - return "{" + this.props1 + " / " + this.props2 + "}"; - } - } - - public static final class LocalPropertiesPair { - - private final RequestedLocalProperties props1, props2; - - public LocalPropertiesPair( - RequestedLocalProperties props1, RequestedLocalProperties props2) { - this.props1 = props1; - this.props2 = props2; - } - - public RequestedLocalProperties getProperties1() { - return this.props1; - } - - public RequestedLocalProperties getProperties2() { - return this.props2; - } - - @Override - public int hashCode() { - return (this.props1 == null ? 0 : this.props1.hashCode()) - ^ (this.props2 == null ? 0 : this.props2.hashCode()); - } - - @Override - public boolean equals(Object obj) { - if (obj.getClass() == LocalPropertiesPair.class) { - final LocalPropertiesPair other = (LocalPropertiesPair) obj; - - return (this.props1 == null - ? other.props1 == null - : this.props1.equals(other.props1)) - && (this.props2 == null - ? other.props2 == null - : this.props2.equals(other.props2)); - } - return false; - } - - @Override - public String toString() { - return "{" + this.props1 + " / " + this.props2 + "}"; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java deleted file mode 100644 index 4a1ef3587d8d2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java +++ /dev/null @@ -1,104 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; - -import java.util.List; - -/** - * Abstract base class for Operator descriptions which instantiates the node and sets the driver - * strategy and the sorting and grouping keys. Returns possible local and global properties and - * updates them after the operation has been performed. - * - * @see org.apache.flink.optimizer.dag.SingleInputNode - */ -public abstract class OperatorDescriptorSingle implements AbstractOperatorDescriptor { - - protected final FieldSet keys; // the set of key fields - protected final FieldList keyList; // the key fields with ordered field positions - - private List globalProps; - private List localProps; - - protected OperatorDescriptorSingle() { - this(null); - } - - protected OperatorDescriptorSingle(FieldSet keys) { - this.keys = keys; - this.keyList = keys == null ? null : keys.toFieldList(); - } - - public List getPossibleGlobalProperties() { - if (this.globalProps == null) { - this.globalProps = createPossibleGlobalProperties(); - } - return this.globalProps; - } - - public List getPossibleLocalProperties() { - if (this.localProps == null) { - this.localProps = createPossibleLocalProperties(); - } - return this.localProps; - } - - /** - * Returns a list of global properties that are required by this operator descriptor. - * - * @return A list of global properties that are required by this operator descriptor. - */ - protected abstract List createPossibleGlobalProperties(); - - /** - * Returns a list of local properties that are required by this operator descriptor. - * - * @return A list of local properties that are required by this operator descriptor. - */ - protected abstract List createPossibleLocalProperties(); - - public abstract SingleInputPlanNode instantiate(Channel in, SingleInputNode node); - - /** - * Returns the global properties which are present after the operator was applied on the - * provided global properties. - * - * @param in The global properties on which the operator is applied. - * @return The global properties which are valid after the operator has been applied. - */ - public abstract GlobalProperties computeGlobalProperties(GlobalProperties in); - - /** - * Returns the local properties which are present after the operator was applied on the provided - * local properties. - * - * @param in The local properties on which the operator is applied. - * @return The local properties which are valid after the operator has been applied. - */ - public abstract LocalProperties computeLocalProperties(LocalProperties in); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java deleted file mode 100644 index 0b402ebd3cbb3..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/PartialGroupProperties.java +++ /dev/null @@ -1,97 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.dag.GroupReduceNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -public final class PartialGroupProperties extends OperatorDescriptorSingle { - - public PartialGroupProperties(FieldSet keys) { - super(keys); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.SORTED_GROUP_COMBINE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - // create in input node for combine with the same parallelism as input node - GroupReduceNode combinerNode = - new GroupReduceNode((GroupReduceOperatorBase) node.getOperator()); - combinerNode.setParallelism(in.getSource().getParallelism()); - - SingleInputPlanNode combiner = - new SingleInputPlanNode( - combinerNode, - "Combine (" + node.getOperator().getName() + ")", - in, - DriverStrategy.SORTED_GROUP_COMBINE); - // sorting key info - combiner.setDriverKeyInfo(in.getLocalStrategyKeys(), in.getLocalStrategySortOrder(), 0); - // set grouping comparator key info - combiner.setDriverKeyInfo(this.keyList, 1); - - return combiner; - } - - @Override - protected List createPossibleGlobalProperties() { - return Collections.singletonList(new RequestedGlobalProperties()); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties props = new RequestedLocalProperties(); - props.setGroupedFields(this.keys); - return Collections.singletonList(props); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java deleted file mode 100644 index 1b89dc2781827..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/ReduceProperties.java +++ /dev/null @@ -1,153 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.PartitionNode; -import org.apache.flink.optimizer.dag.ReduceNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.List; - -public final class ReduceProperties extends OperatorDescriptorSingle { - private static final Logger LOG = LoggerFactory.getLogger(ReduceProperties.class); - - private final Partitioner customPartitioner; - - private final DriverStrategy combinerStrategy; - - public ReduceProperties(FieldSet keys, DriverStrategy combinerStrategy) { - this(keys, null, combinerStrategy); - } - - public ReduceProperties( - FieldSet keys, Partitioner customPartitioner, DriverStrategy combinerStrategy) { - super(keys); - this.customPartitioner = customPartitioner; - this.combinerStrategy = combinerStrategy; - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.SORTED_REDUCE; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - Channel toReducer = in; - - if (in.getShipStrategy() == ShipStrategyType.FORWARD - || (node.getBroadcastConnections() != null - && !node.getBroadcastConnections().isEmpty())) { - if (in.getSource().getOptimizerNode() instanceof PartitionNode) { - LOG.warn( - "Cannot automatically inject combiner for ReduceFunction. Please add an explicit combiner with combineGroup() in front of the partition operator."); - } - } else if (combinerStrategy != DriverStrategy.NONE) { - // non forward case. all local properties are killed anyways, so we can safely plug in a - // combiner - Channel toCombiner = new Channel(in.getSource()); - toCombiner.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - - // create an input node for combine with same parallelism as input node - ReduceNode combinerNode = ((ReduceNode) node).getCombinerUtilityNode(); - combinerNode.setParallelism(in.getSource().getParallelism()); - - SingleInputPlanNode combiner = - new SingleInputPlanNode( - combinerNode, - "Combine (" + node.getOperator().getName() + ")", - toCombiner, - this.combinerStrategy, - this.keyList); - - combiner.setCosts(new Costs(0, 0)); - combiner.initProperties( - toCombiner.getGlobalProperties(), toCombiner.getLocalProperties()); - - toReducer = new Channel(combiner); - toReducer.setShipStrategy( - in.getShipStrategy(), - in.getShipStrategyKeys(), - in.getShipStrategySortOrder(), - in.getDataExchangeMode()); - toReducer.setLocalStrategy( - LocalStrategy.SORT, in.getLocalStrategyKeys(), in.getLocalStrategySortOrder()); - } - - return new SingleInputPlanNode( - node, - "Reduce (" + node.getOperator().getName() + ")", - toReducer, - DriverStrategy.SORTED_REDUCE, - this.keyList); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties props = new RequestedGlobalProperties(); - if (customPartitioner == null) { - props.setAnyPartitioning(this.keys); - } else { - props.setCustomPartitioned(this.keys, this.customPartitioner); - } - return Collections.singletonList(props); - } - - @Override - protected List createPossibleLocalProperties() { - RequestedLocalProperties props = new RequestedLocalProperties(); - props.setGroupedFields(this.keys); - return Collections.singletonList(props); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - if (gProps.getUniqueFieldCombination() != null - && gProps.getUniqueFieldCombination().size() > 0 - && gProps.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - gProps.setAnyPartitioning( - gProps.getUniqueFieldCombination().iterator().next().toFieldList()); - } - gProps.clearUniqueFieldCombinations(); - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps.clearUniqueFieldSets(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SolutionSetDeltaOperator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SolutionSetDeltaOperator.java deleted file mode 100644 index 04513fa44f6b7..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SolutionSetDeltaOperator.java +++ /dev/null @@ -1,72 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** */ -public class SolutionSetDeltaOperator extends OperatorDescriptorSingle { - - public SolutionSetDeltaOperator(FieldList partitioningFields) { - super(partitioningFields); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.UNARY_NO_OP; - } - - @Override - public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) { - return new SingleInputPlanNode(node, "SolutionSet Delta", in, DriverStrategy.UNARY_NO_OP); - } - - @Override - protected List createPossibleGlobalProperties() { - RequestedGlobalProperties partProps = new RequestedGlobalProperties(); - partProps.setHashPartitioned(this.keyList); - return Collections.singletonList(partProps); - } - - @Override - protected List createPossibleLocalProperties() { - return Collections.singletonList(new RequestedLocalProperties()); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties gProps) { - return gProps; - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties lProps) { - return lProps; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java deleted file mode 100644 index 17c3ad0211052..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeFullOuterJoinDescriptor.java +++ /dev/null @@ -1,39 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class SortMergeFullOuterJoinDescriptor extends AbstractSortMergeJoinDescriptor { - - public SortMergeFullOuterJoinDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2, false, false, true); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.FULL_OUTER_MERGE; - } - - @Override - protected String getNodeName() { - return "FullOuterJoin"; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java deleted file mode 100644 index 7711b45dbd86d..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeInnerJoinDescriptor.java +++ /dev/null @@ -1,48 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class SortMergeInnerJoinDescriptor extends AbstractSortMergeJoinDescriptor { - - public SortMergeInnerJoinDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2); - } - - public SortMergeInnerJoinDescriptor( - FieldList keys1, - FieldList keys2, - boolean broadcastFirstAllowed, - boolean broadcastSecondAllowed, - boolean repartitionAllowed) { - super(keys1, keys2, broadcastFirstAllowed, broadcastSecondAllowed, repartitionAllowed); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.INNER_MERGE; - } - - @Override - protected String getNodeName() { - return "Join"; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java deleted file mode 100644 index e159db4954203..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeLeftOuterJoinDescriptor.java +++ /dev/null @@ -1,44 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class SortMergeLeftOuterJoinDescriptor extends AbstractSortMergeJoinDescriptor { - - public SortMergeLeftOuterJoinDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2, false, true, true); - } - - public SortMergeLeftOuterJoinDescriptor( - FieldList keys1, FieldList keys2, boolean broadcastAllowed) { - super(keys1, keys2, false, broadcastAllowed, true); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.LEFT_OUTER_MERGE; - } - - @Override - protected String getNodeName() { - return "LeftOuterJoin"; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java deleted file mode 100644 index 5e79f43ee38a8..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/SortMergeRightOuterJoinDescriptor.java +++ /dev/null @@ -1,44 +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.optimizer.operators; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.runtime.operators.DriverStrategy; - -public class SortMergeRightOuterJoinDescriptor extends AbstractSortMergeJoinDescriptor { - - public SortMergeRightOuterJoinDescriptor(FieldList keys1, FieldList keys2) { - super(keys1, keys2, true, false, true); - } - - public SortMergeRightOuterJoinDescriptor( - FieldList keys1, FieldList keys2, boolean broadcastAllowed) { - super(keys1, keys2, broadcastAllowed, false, true); - } - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.RIGHT_OUTER_MERGE; - } - - @Override - protected String getNodeName() { - return "RightOuterJoin"; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/UtilSinkJoinOpDescriptor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/UtilSinkJoinOpDescriptor.java deleted file mode 100644 index 683b268af318b..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/UtilSinkJoinOpDescriptor.java +++ /dev/null @@ -1,96 +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.optimizer.operators; - -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.SinkJoiner; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.SinkJoinerPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Collections; -import java.util.List; - -/** */ -public class UtilSinkJoinOpDescriptor extends OperatorDescriptorDual { - - @Override - public DriverStrategy getStrategy() { - return DriverStrategy.BINARY_NO_OP; - } - - @Override - protected List createPossibleGlobalProperties() { - // all properties are possible - return Collections.singletonList( - new GlobalPropertiesPair( - new RequestedGlobalProperties(), new RequestedGlobalProperties())); - } - - @Override - protected List createPossibleLocalProperties() { - // all properties are possible - return Collections.singletonList( - new LocalPropertiesPair( - new RequestedLocalProperties(), new RequestedLocalProperties())); - } - - @Override - public boolean areCompatible( - RequestedGlobalProperties requested1, - RequestedGlobalProperties requested2, - GlobalProperties produced1, - GlobalProperties produced2) { - return true; - } - - @Override - public boolean areCoFulfilled( - RequestedLocalProperties requested1, - RequestedLocalProperties requested2, - LocalProperties produced1, - LocalProperties produced2) { - return true; - } - - @Override - public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { - if (node instanceof SinkJoiner) { - return new SinkJoinerPlanNode((SinkJoiner) node, in1, in2); - } else { - throw new CompilerException(); - } - } - - @Override - public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { - return new LocalProperties(); - } - - @Override - public GlobalProperties computeGlobalProperties(GlobalProperties in1, GlobalProperties in2) { - return GlobalProperties.combine(in1, in2); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BinaryUnionPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BinaryUnionPlanNode.java deleted file mode 100644 index 2f42a005d08fb..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BinaryUnionPlanNode.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.optimizer.plan; - -import org.apache.flink.optimizer.dag.BinaryUnionNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -/** A special subclass for the union to make it identifiable. */ -public class BinaryUnionPlanNode extends DualInputPlanNode { - - /** @param template */ - public BinaryUnionPlanNode(BinaryUnionNode template, Channel in1, Channel in2) { - super(template, "Union", in1, in2, DriverStrategy.UNION); - } - - public BinaryUnionPlanNode(BinaryUnionPlanNode toSwapFrom) { - super( - toSwapFrom.getOptimizerNode(), - "Union-With-Cached", - toSwapFrom.getInput2(), - toSwapFrom.getInput1(), - DriverStrategy.UNION_WITH_CACHED); - - this.globalProps = toSwapFrom.globalProps; - this.localProps = toSwapFrom.localProps; - this.nodeCosts = toSwapFrom.nodeCosts; - this.cumulativeCosts = toSwapFrom.cumulativeCosts; - - setParallelism(toSwapFrom.getParallelism()); - } - - public BinaryUnionNode getOptimizerNode() { - return (BinaryUnionNode) this.template; - } - - public boolean unionsStaticAndDynamicPath() { - return getInput1().isOnDynamicPath() != getInput2().isOnDynamicPath(); - } - - @Override - public int getMemoryConsumerWeight() { - return 0; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BulkIterationPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BulkIterationPlanNode.java deleted file mode 100644 index 762ae667fa838..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BulkIterationPlanNode.java +++ /dev/null @@ -1,172 +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.optimizer.plan; - -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.BulkIterationNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.HashMap; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; - -public class BulkIterationPlanNode extends SingleInputPlanNode implements IterationPlanNode { - - private final BulkPartialSolutionPlanNode partialSolutionPlanNode; - - private final PlanNode rootOfStepFunction; - - private PlanNode rootOfTerminationCriterion; - - private TypeSerializerFactory serializerForIterationChannel; - - // -------------------------------------------------------------------------------------------- - - public BulkIterationPlanNode( - BulkIterationNode template, - String nodeName, - Channel input, - BulkPartialSolutionPlanNode pspn, - PlanNode rootOfStepFunction) { - super(template, nodeName, input, DriverStrategy.NONE); - this.partialSolutionPlanNode = pspn; - this.rootOfStepFunction = rootOfStepFunction; - - mergeBranchPlanMaps(); - } - - public BulkIterationPlanNode( - BulkIterationNode template, - String nodeName, - Channel input, - BulkPartialSolutionPlanNode pspn, - PlanNode rootOfStepFunction, - PlanNode rootOfTerminationCriterion) { - this(template, nodeName, input, pspn, rootOfStepFunction); - this.rootOfTerminationCriterion = rootOfTerminationCriterion; - } - - // -------------------------------------------------------------------------------------------- - - public BulkIterationNode getIterationNode() { - if (this.template instanceof BulkIterationNode) { - return (BulkIterationNode) this.template; - } else { - throw new RuntimeException(); - } - } - - public BulkPartialSolutionPlanNode getPartialSolutionPlanNode() { - return this.partialSolutionPlanNode; - } - - public PlanNode getRootOfStepFunction() { - return this.rootOfStepFunction; - } - - public PlanNode getRootOfTerminationCriterion() { - return this.rootOfTerminationCriterion; - } - - // -------------------------------------------------------------------------------------------- - - public TypeSerializerFactory getSerializerForIterationChannel() { - return serializerForIterationChannel; - } - - public void setSerializerForIterationChannel( - TypeSerializerFactory serializerForIterationChannel) { - this.serializerForIterationChannel = serializerForIterationChannel; - } - - public void setCosts(Costs nodeCosts) { - // add the costs from the step function - nodeCosts.addCosts(this.rootOfStepFunction.getCumulativeCosts()); - - // add the costs for the termination criterion, if it exists - // the costs are divided at branches, so we can simply add them up - if (rootOfTerminationCriterion != null) { - nodeCosts.addCosts(this.rootOfTerminationCriterion.getCumulativeCosts()); - } - - super.setCosts(nodeCosts); - } - - public int getMemoryConsumerWeight() { - return 1; - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } - - SourceAndDamReport fromOutside = super.hasDamOnPathDownTo(source); - - if (fromOutside == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (fromOutside == FOUND_SOURCE) { - // we always have a dam in the back channel - return FOUND_SOURCE_AND_DAM; - } else { - // check the step function for dams - return this.rootOfStepFunction.hasDamOnPathDownTo(source); - } - } - - @Override - public void acceptForStepFunction(Visitor visitor) { - this.rootOfStepFunction.accept(visitor); - - if (this.rootOfTerminationCriterion != null) { - this.rootOfTerminationCriterion.accept(visitor); - } - } - - private void mergeBranchPlanMaps() { - for (OptimizerNode.UnclosedBranchDescriptor desc : template.getOpenBranches()) { - OptimizerNode brancher = desc.getBranchingNode(); - - if (branchPlan == null) { - branchPlan = new HashMap(6); - } - - if (!branchPlan.containsKey(brancher)) { - PlanNode selectedCandidate = null; - - if (rootOfStepFunction.branchPlan != null) { - selectedCandidate = rootOfStepFunction.branchPlan.get(brancher); - } - - if (selectedCandidate == null) { - throw new CompilerException( - "Candidates for a node with open branches are missing information about the selected candidate "); - } - - this.branchPlan.put(brancher, selectedCandidate); - } - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BulkPartialSolutionPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BulkPartialSolutionPlanNode.java deleted file mode 100644 index c108acec7879a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/BulkPartialSolutionPlanNode.java +++ /dev/null @@ -1,125 +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.optimizer.plan; - -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.BulkPartialSolutionNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DamBehavior; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.Collections; -import java.util.HashMap; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** Plan candidate node for partial solution of a bulk iteration. */ -public class BulkPartialSolutionPlanNode extends PlanNode { - - private static final Costs NO_COSTS = new Costs(); - - private BulkIterationPlanNode containingIterationNode; - - private Channel initialInput; - - public Object postPassHelper; - - public BulkPartialSolutionPlanNode( - BulkPartialSolutionNode template, - String nodeName, - GlobalProperties gProps, - LocalProperties lProps, - Channel initialInput) { - super(template, nodeName, DriverStrategy.NONE); - - this.globalProps = gProps; - this.localProps = lProps; - this.initialInput = initialInput; - - // the partial solution does not cost anything - this.nodeCosts = NO_COSTS; - this.cumulativeCosts = NO_COSTS; - - if (initialInput.getSource().branchPlan != null - && initialInput.getSource().branchPlan.size() > 0) { - if (this.branchPlan == null) { - this.branchPlan = new HashMap(); - } - - this.branchPlan.putAll(initialInput.getSource().branchPlan); - } - } - - // -------------------------------------------------------------------------------------------- - - public BulkPartialSolutionNode getPartialSolutionNode() { - return (BulkPartialSolutionNode) this.template; - } - - public BulkIterationPlanNode getContainingIterationNode() { - return this.containingIterationNode; - } - - public void setContainingIterationNode(BulkIterationPlanNode containingIterationNode) { - this.containingIterationNode = containingIterationNode; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - visitor.postVisit(this); - } - } - - @Override - public Iterable getPredecessors() { - return Collections.emptyList(); - } - - @Override - public Iterable getInputs() { - return Collections.emptyList(); - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } - SourceAndDamReport res = this.initialInput.getSource().hasDamOnPathDownTo(source); - if (res == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (res == FOUND_SOURCE) { - return (this.initialInput.getLocalStrategy().dams() - || this.initialInput.getTempMode().breaksPipeline() - || getDriverStrategy().firstDam() == DamBehavior.FULL_DAM) - ? FOUND_SOURCE_AND_DAM - : FOUND_SOURCE; - } else { - return NOT_FOUND; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java deleted file mode 100644 index 86b7b0560d490..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java +++ /dev/null @@ -1,571 +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.optimizer.plan; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.EstimateProvider; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plandump.DumpableConnection; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A Channel represents the result produced by an operator and the data exchange before the - * consumption by the target operator. - * - *

The channel defines and tracks various properties and characteristics of the data set and data - * exchange. - * - *

Data set characteristics: - * - *

    - *
  • The "global properties" of the data, i.e., how the data is distributed across partitions - *
  • The "required global properties" of the data, i.e., the global properties that, if absent, - * would cause the program to return a wrong result. - *
  • The "local properties" of the data, i.e., how the data is organized within a partition - *
  • The "required local properties" of the data, i.e., the local properties that, if absent, - * would cause the program to return a wrong result. - *
- * - * Data exchange parameters: - * - *
    - *
  • The "ship strategy", i.e., whether to forward the data, shuffle it, broadcast it, ... - *
  • The "ship keys", which are the positions of the key fields in the exchanged records. - *
  • The "data exchange mode", which defines whether to pipeline or batch the exchange - *
  • Several more... - *
- */ -public class Channel implements EstimateProvider, Cloneable, DumpableConnection { - - private PlanNode source; - - private PlanNode target; - - private ShipStrategyType shipStrategy = ShipStrategyType.NONE; - - private DataExchangeMode dataExchangeMode; - - private LocalStrategy localStrategy = LocalStrategy.NONE; - - private FieldList shipKeys; - - private FieldList localKeys; - - private boolean[] shipSortOrder; - - private boolean[] localSortOrder; - - private RequestedGlobalProperties requiredGlobalProps; - - private RequestedLocalProperties requiredLocalProps; - - private GlobalProperties globalProps; - - private LocalProperties localProps; - - private TypeSerializerFactory serializer; - - private TypeComparatorFactory shipStrategyComparator; - - private TypeComparatorFactory localStrategyComparator; - - private DataDistribution dataDistribution; - - private Partitioner partitioner; - - private TempMode tempMode; - - private double relativeTempMemory; - - private double relativeMemoryLocalStrategy; - - private int replicationFactor = 1; - - // -------------------------------------------------------------------------------------------- - - public Channel(PlanNode sourceNode) { - this(sourceNode, null); - } - - public Channel(PlanNode sourceNode, TempMode tempMode) { - this.source = sourceNode; - this.tempMode = (tempMode == null ? TempMode.NONE : tempMode); - } - - // -------------------------------------------------------------------------------------------- - // Accessors - // -------------------------------------------------------------------------------------------- - - /** - * Gets the source of this Channel. - * - * @return The source. - */ - @Override - public PlanNode getSource() { - return this.source; - } - - public void setSource(PlanNode source) { - this.source = source; - } - - /** - * Sets the target of this Channel. - * - * @param target The target. - */ - public void setTarget(PlanNode target) { - this.target = target; - } - - /** - * Gets the target of this Channel. - * - * @return The target. - */ - public PlanNode getTarget() { - return this.target; - } - - public void setShipStrategy(ShipStrategyType strategy, DataExchangeMode dataExchangeMode) { - setShipStrategy(strategy, null, null, null, dataExchangeMode); - } - - public void setShipStrategy( - ShipStrategyType strategy, FieldList keys, DataExchangeMode dataExchangeMode) { - setShipStrategy(strategy, keys, null, null, dataExchangeMode); - } - - public void setShipStrategy( - ShipStrategyType strategy, - FieldList keys, - boolean[] sortDirection, - DataExchangeMode dataExchangeMode) { - setShipStrategy(strategy, keys, sortDirection, null, dataExchangeMode); - } - - public void setShipStrategy( - ShipStrategyType strategy, - FieldList keys, - Partitioner partitioner, - DataExchangeMode dataExchangeMode) { - setShipStrategy(strategy, keys, null, partitioner, dataExchangeMode); - } - - public void setShipStrategy( - ShipStrategyType strategy, - FieldList keys, - boolean[] sortDirection, - Partitioner partitioner, - DataExchangeMode dataExchangeMode) { - this.shipStrategy = strategy; - this.shipKeys = keys; - this.shipSortOrder = sortDirection; - this.partitioner = partitioner; - this.dataExchangeMode = dataExchangeMode; - this.globalProps = null; // reset the global properties - } - - /** - * Sets the data exchange mode (batch / pipelined) to use for the data exchange of this channel. - */ - public void setDataExchangeMode(DataExchangeMode dataExchangeMode) { - this.dataExchangeMode = checkNotNull(dataExchangeMode); - } - - /** - * Gets the data exchange mode (batch / pipelined) to use for the data exchange of this channel. - * - * @return The data exchange mode of this channel. - */ - public DataExchangeMode getDataExchangeMode() { - return dataExchangeMode; - } - - public ShipStrategyType getShipStrategy() { - return this.shipStrategy; - } - - public FieldList getShipStrategyKeys() { - return this.shipKeys; - } - - public boolean[] getShipStrategySortOrder() { - return this.shipSortOrder; - } - - public void setLocalStrategy(LocalStrategy strategy) { - setLocalStrategy(strategy, null, null); - } - - public void setLocalStrategy(LocalStrategy strategy, FieldList keys, boolean[] sortDirection) { - this.localStrategy = strategy; - this.localKeys = keys; - this.localSortOrder = sortDirection; - this.localProps = null; // reset the local properties - } - - public LocalStrategy getLocalStrategy() { - return this.localStrategy; - } - - public FieldList getLocalStrategyKeys() { - return this.localKeys; - } - - public boolean[] getLocalStrategySortOrder() { - return this.localSortOrder; - } - - public void setDataDistribution(DataDistribution dataDistribution) { - this.dataDistribution = dataDistribution; - } - - public DataDistribution getDataDistribution() { - return this.dataDistribution; - } - - public Partitioner getPartitioner() { - return partitioner; - } - - public TempMode getTempMode() { - return this.tempMode; - } - - /** - * Sets the temp mode of the connection. - * - * @param tempMode The temp mode of the connection. - */ - public void setTempMode(TempMode tempMode) { - this.tempMode = tempMode; - } - - /** - * Gets the memory for materializing the channel's result from this Channel. - * - * @return The temp memory. - */ - public double getRelativeTempMemory() { - return this.relativeTempMemory; - } - - /** - * Sets the memory for materializing the channel's result from this Channel. - * - * @param relativeTempMemory The memory for materialization. - */ - public void setRelativeTempMemory(double relativeTempMemory) { - this.relativeTempMemory = relativeTempMemory; - } - - /** - * Sets the replication factor of the connection. - * - * @param factor The replication factor of the connection. - */ - public void setReplicationFactor(int factor) { - this.replicationFactor = factor; - } - - /** - * Returns the replication factor of the connection. - * - * @return The replication factor of the connection. - */ - public int getReplicationFactor() { - return this.replicationFactor; - } - - /** - * Gets the serializer from this Channel. - * - * @return The serializer. - */ - public TypeSerializerFactory getSerializer() { - return serializer; - } - - /** - * Sets the serializer for this Channel. - * - * @param serializer The serializer to set. - */ - public void setSerializer(TypeSerializerFactory serializer) { - this.serializer = serializer; - } - - /** - * Gets the ship strategy comparator from this Channel. - * - * @return The ship strategy comparator. - */ - public TypeComparatorFactory getShipStrategyComparator() { - return shipStrategyComparator; - } - - /** - * Sets the ship strategy comparator for this Channel. - * - * @param shipStrategyComparator The ship strategy comparator to set. - */ - public void setShipStrategyComparator(TypeComparatorFactory shipStrategyComparator) { - this.shipStrategyComparator = shipStrategyComparator; - } - - /** - * Gets the local strategy comparator from this Channel. - * - * @return The local strategy comparator. - */ - public TypeComparatorFactory getLocalStrategyComparator() { - return localStrategyComparator; - } - - /** - * Sets the local strategy comparator for this Channel. - * - * @param localStrategyComparator The local strategy comparator to set. - */ - public void setLocalStrategyComparator(TypeComparatorFactory localStrategyComparator) { - this.localStrategyComparator = localStrategyComparator; - } - - public double getRelativeMemoryLocalStrategy() { - return relativeMemoryLocalStrategy; - } - - public void setRelativeMemoryLocalStrategy(double relativeMemoryLocalStrategy) { - this.relativeMemoryLocalStrategy = relativeMemoryLocalStrategy; - } - - public boolean isOnDynamicPath() { - return this.source.isOnDynamicPath(); - } - - public int getCostWeight() { - return this.source.getCostWeight(); - } - - // -------------------------------------------------------------------------------------------- - // Statistic Estimates - // -------------------------------------------------------------------------------------------- - - @Override - public long getEstimatedOutputSize() { - long estimate = this.source.template.getEstimatedOutputSize(); - return estimate < 0 ? estimate : estimate * this.replicationFactor; - } - - @Override - public long getEstimatedNumRecords() { - long estimate = this.source.template.getEstimatedNumRecords(); - return estimate < 0 ? estimate : estimate * this.replicationFactor; - } - - @Override - public float getEstimatedAvgWidthPerOutputRecord() { - return this.source.template.getEstimatedAvgWidthPerOutputRecord(); - } - - // -------------------------------------------------------------------------------------------- - // Data Property Handling - // -------------------------------------------------------------------------------------------- - - public RequestedGlobalProperties getRequiredGlobalProps() { - return requiredGlobalProps; - } - - public void setRequiredGlobalProps(RequestedGlobalProperties requiredGlobalProps) { - this.requiredGlobalProps = requiredGlobalProps; - } - - public RequestedLocalProperties getRequiredLocalProps() { - return requiredLocalProps; - } - - public void setRequiredLocalProps(RequestedLocalProperties requiredLocalProps) { - this.requiredLocalProps = requiredLocalProps; - } - - public GlobalProperties getGlobalProperties() { - if (this.globalProps == null) { - this.globalProps = this.source.getGlobalProperties().clone(); - switch (this.shipStrategy) { - case BROADCAST: - this.globalProps.clearUniqueFieldCombinations(); - this.globalProps.setFullyReplicated(); - break; - case PARTITION_HASH: - this.globalProps.setHashPartitioned(this.shipKeys); - break; - case PARTITION_RANGE: - this.globalProps.setRangePartitioned( - Utils.createOrdering(this.shipKeys, this.shipSortOrder), - this.dataDistribution); - break; - case FORWARD: - break; - case PARTITION_RANDOM: - this.globalProps.reset(); - break; - case PARTITION_FORCED_REBALANCE: - this.globalProps.setForcedRebalanced(); - break; - case PARTITION_CUSTOM: - this.globalProps.setCustomPartitioned(this.shipKeys, this.partitioner); - break; - case NONE: - throw new CompilerException( - "Cannot produce GlobalProperties before ship strategy is set."); - } - } - - return this.globalProps; - } - - public LocalProperties getLocalProperties() { - if (this.localProps == null) { - computeLocalPropertiesAfterShippingOnly(); - switch (this.localStrategy) { - case NONE: - break; - case SORT: - case COMBININGSORT: - this.localProps = - LocalProperties.forOrdering( - Utils.createOrdering(this.localKeys, this.localSortOrder)); - break; - default: - throw new CompilerException("Unsupported local strategy for channel."); - } - } - - return this.localProps; - } - - private void computeLocalPropertiesAfterShippingOnly() { - switch (this.shipStrategy) { - case BROADCAST: - case PARTITION_HASH: - case PARTITION_CUSTOM: - case PARTITION_RANGE: - case PARTITION_RANDOM: - case PARTITION_FORCED_REBALANCE: - this.localProps = new LocalProperties(); - break; - case FORWARD: - this.localProps = this.source.getLocalProperties(); - break; - case NONE: - throw new CompilerException("ShipStrategy has not yet been set."); - default: - throw new CompilerException("Unknown ShipStrategy."); - } - } - - public void adjustGlobalPropertiesForFullParallelismChange() { - if (this.shipStrategy == null || this.shipStrategy == ShipStrategyType.NONE) { - throw new IllegalStateException( - "Cannot adjust channel for parallelism " - + "change before the ship strategy is set."); - } - - // make sure the properties are acquired - if (this.globalProps == null) { - getGlobalProperties(); - } - - // some strategies globally reestablish properties - switch (this.shipStrategy) { - case FORWARD: - throw new CompilerException( - "Cannot use FORWARD strategy between operations " - + "with different number of parallel instances."); - case NONE: // excluded by sanity check. left here for verification check completion - case BROADCAST: - case PARTITION_HASH: - case PARTITION_RANGE: - case PARTITION_RANDOM: - case PARTITION_FORCED_REBALANCE: - case PARTITION_CUSTOM: - return; - } - throw new CompilerException("Unrecognized Ship Strategy Type: " + this.shipStrategy); - } - - // -------------------------------------------------------------------------------------------- - - /** Utility method used while swapping binary union nodes for n-ary union nodes. */ - public void swapUnionNodes(PlanNode newUnionNode) { - if (!(this.source instanceof BinaryUnionPlanNode)) { - throw new IllegalStateException(); - } else { - this.source = newUnionNode; - } - } - - // -------------------------------------------------------------------------------------------- - - public int getMaxDepth() { - return this.source.getOptimizerNode().getMaxDepth() + 1; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public String toString() { - return "Channel (" - + this.source - + (this.target == null ? ')' : ") -> (" + this.target + ')') - + '[' - + this.shipStrategy - + "] [" - + this.localStrategy - + "] " - + (this.tempMode == null || this.tempMode == TempMode.NONE - ? "{NO-TEMP}" - : this.tempMode); - } - - @Override - public Channel clone() { - try { - return (Channel) super.clone(); - } catch (CloneNotSupportedException cnsex) { - throw new RuntimeException(cnsex); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/DualInputPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/DualInputPlanNode.java deleted file mode 100644 index 9b96a44c4c18a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/DualInputPlanNode.java +++ /dev/null @@ -1,261 +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.optimizer.plan; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.runtime.operators.DamBehavior; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** */ -public class DualInputPlanNode extends PlanNode { - - protected final Channel input1; - protected final Channel input2; - - protected final FieldList keys1; - protected final FieldList keys2; - - protected final boolean[] sortOrders; - - private TypeComparatorFactory comparator1; - private TypeComparatorFactory comparator2; - private TypePairComparatorFactory pairComparator; - - public Object postPassHelper1; - public Object postPassHelper2; - - // -------------------------------------------------------------------------------------------- - - public DualInputPlanNode( - OptimizerNode template, - String nodeName, - Channel input1, - Channel input2, - DriverStrategy diverStrategy) { - this(template, nodeName, input1, input2, diverStrategy, null, null, null); - } - - public DualInputPlanNode( - OptimizerNode template, - String nodeName, - Channel input1, - Channel input2, - DriverStrategy diverStrategy, - FieldList driverKeyFields1, - FieldList driverKeyFields2) { - this( - template, - nodeName, - input1, - input2, - diverStrategy, - driverKeyFields1, - driverKeyFields2, - SingleInputPlanNode.getTrueArray(driverKeyFields1.size())); - } - - public DualInputPlanNode( - OptimizerNode template, - String nodeName, - Channel input1, - Channel input2, - DriverStrategy diverStrategy, - FieldList driverKeyFields1, - FieldList driverKeyFields2, - boolean[] driverSortOrders) { - super(template, nodeName, diverStrategy); - this.input1 = input1; - this.input2 = input2; - this.keys1 = driverKeyFields1; - this.keys2 = driverKeyFields2; - this.sortOrders = driverSortOrders; - - if (this.input1.getShipStrategy() == ShipStrategyType.BROADCAST) { - this.input1.setReplicationFactor(getParallelism()); - } - if (this.input2.getShipStrategy() == ShipStrategyType.BROADCAST) { - this.input2.setReplicationFactor(getParallelism()); - } - - mergeBranchPlanMaps(input1.getSource(), input2.getSource()); - } - - // -------------------------------------------------------------------------------------------- - - public TwoInputNode getTwoInputNode() { - if (this.template instanceof TwoInputNode) { - return (TwoInputNode) this.template; - } else { - throw new RuntimeException(); - } - } - - public FieldList getKeysForInput1() { - return this.keys1; - } - - public FieldList getKeysForInput2() { - return this.keys2; - } - - public boolean[] getSortOrders() { - return this.sortOrders; - } - - public TypeComparatorFactory getComparator1() { - return this.comparator1; - } - - public TypeComparatorFactory getComparator2() { - return this.comparator2; - } - - public void setComparator1(TypeComparatorFactory comparator) { - this.comparator1 = comparator; - } - - public void setComparator2(TypeComparatorFactory comparator) { - this.comparator2 = comparator; - } - - public TypePairComparatorFactory getPairComparator() { - return this.pairComparator; - } - - public void setPairComparator(TypePairComparatorFactory comparator) { - this.pairComparator = comparator; - } - - /** - * Gets the first input channel to this node. - * - * @return The first input channel to this node. - */ - public Channel getInput1() { - return this.input1; - } - - /** - * Gets the second input channel to this node. - * - * @return The second input channel to this node. - */ - public Channel getInput2() { - return this.input2; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - this.input1.getSource().accept(visitor); - this.input2.getSource().accept(visitor); - - for (Channel broadcastInput : getBroadcastInputs()) { - broadcastInput.getSource().accept(visitor); - } - - visitor.postVisit(this); - } - } - - @Override - public Iterable getPredecessors() { - if (getBroadcastInputs() == null || getBroadcastInputs().isEmpty()) { - return Arrays.asList(this.input1.getSource(), this.input2.getSource()); - } else { - List preds = new ArrayList(); - - preds.add(input1.getSource()); - preds.add(input2.getSource()); - - for (Channel c : getBroadcastInputs()) { - preds.add(c.getSource()); - } - - return preds; - } - } - - @Override - public Iterable getInputs() { - return Arrays.asList(this.input1, this.input2); - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } - - // check first input - SourceAndDamReport res1 = this.input1.getSource().hasDamOnPathDownTo(source); - if (res1 == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (res1 == FOUND_SOURCE) { - if (this.input1.getLocalStrategy().dams() - || this.input1.getTempMode().breaksPipeline() - || getDriverStrategy().firstDam() == DamBehavior.FULL_DAM) { - return FOUND_SOURCE_AND_DAM; - } else { - return FOUND_SOURCE; - } - } else { - SourceAndDamReport res2 = this.input2.getSource().hasDamOnPathDownTo(source); - if (res2 == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (res2 == FOUND_SOURCE) { - if (this.input2.getLocalStrategy().dams() - || this.input2.getTempMode().breaksPipeline() - || getDriverStrategy().secondDam() == DamBehavior.FULL_DAM) { - return FOUND_SOURCE_AND_DAM; - } else { - return FOUND_SOURCE; - } - } else { - // NOT_FOUND - // check the broadcast inputs - - for (NamedChannel nc : getBroadcastInputs()) { - SourceAndDamReport bcRes = nc.getSource().hasDamOnPathDownTo(source); - if (bcRes != NOT_FOUND) { - // broadcast inputs are always dams - return FOUND_SOURCE_AND_DAM; - } - } - return NOT_FOUND; - } - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/IterationPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/IterationPlanNode.java deleted file mode 100644 index a9b767cf654a0..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/IterationPlanNode.java +++ /dev/null @@ -1,30 +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.optimizer.plan; - -import org.apache.flink.optimizer.dag.IterationNode; -import org.apache.flink.util.Visitor; - -/** */ -public interface IterationPlanNode { - - void acceptForStepFunction(Visitor visitor); - - IterationNode getIterationNode(); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/NAryUnionPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/NAryUnionPlanNode.java deleted file mode 100644 index c593fa558330a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/NAryUnionPlanNode.java +++ /dev/null @@ -1,104 +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.optimizer.plan; - -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.BinaryUnionNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.IterableIterator; -import org.apache.flink.util.Visitor; - -import java.util.Collections; -import java.util.Iterator; -import java.util.List; - -/** A union operation over multiple inputs (2 or more). */ -public class NAryUnionPlanNode extends PlanNode { - - private final List inputs; - - /** @param template */ - public NAryUnionPlanNode( - BinaryUnionNode template, - List inputs, - GlobalProperties gProps, - Costs cumulativeCosts) { - super(template, "Union", DriverStrategy.NONE); - - this.inputs = inputs; - this.globalProps = gProps; - this.localProps = new LocalProperties(); - this.nodeCosts = new Costs(); - this.cumulativeCosts = cumulativeCosts; - } - - @Override - public void accept(Visitor visitor) { - visitor.preVisit(this); - for (Channel c : this.inputs) { - c.getSource().accept(visitor); - } - visitor.postVisit(this); - } - - public List getListOfInputs() { - return this.inputs; - } - - @Override - public Iterable getInputs() { - return Collections.unmodifiableList(this.inputs); - } - - @Override - public Iterable getPredecessors() { - final Iterator channels = this.inputs.iterator(); - return new IterableIterator() { - - @Override - public boolean hasNext() { - return channels.hasNext(); - } - - @Override - public PlanNode next() { - return channels.next().getSource(); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - @Override - public Iterator iterator() { - return this; - } - }; - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - // this node is used after the plan enumeration. consequently, this will never be invoked - // here - throw new UnsupportedOperationException(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/NamedChannel.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/NamedChannel.java deleted file mode 100644 index ee0fc05ab66c2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/NamedChannel.java +++ /dev/null @@ -1,45 +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.optimizer.plan; - -import org.apache.flink.optimizer.dag.TempMode; - -public class NamedChannel extends Channel { - - private final String name; - - /** - * Initializes NamedChannel. - * - * @param sourceNode - */ - public NamedChannel(String name, PlanNode sourceNode) { - super(sourceNode); - this.name = name; - } - - public NamedChannel(String name, PlanNode sourceNode, TempMode tempMode) { - super(sourceNode, tempMode); - this.name = name; - } - - public String getName() { - return this.name; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java deleted file mode 100644 index 98c32e7a27179..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java +++ /dev/null @@ -1,133 +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.optimizer.plan; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.util.Visitable; -import org.apache.flink.util.Visitor; - -import java.util.Collection; - -/** - * The execution plan generated by the Optimizer. It contains {@link PlanNode}s and {@link Channel}s - * that describe exactly how the program should be executed. - * - *

The optimized plan defines all ship strategies (local pipe, shuffle, broadcast, rebalance), - * all operator strategies (sorting-merge join, hash join, sorted grouping, ...), and the data - * exchange modes (batched, pipelined). - */ -public class OptimizedPlan implements Visitable { - - /** The data sources in the plan. */ - private final Collection dataSources; - - /** The data sinks in the plan. */ - private final Collection dataSinks; - - /** All nodes in the optimizer plan. */ - private final Collection allNodes; - - /** The original program (as a dataflow plan). */ - private final Plan originalProgram; - - /** Name of the job */ - private final String jobName; - - /** - * Creates a new instance of this optimizer plan container. The plan is given and fully - * described by the data sources, sinks and the collection of all nodes. - * - * @param sources The data sources. - * @param sinks The data sinks. - * @param allNodes A collection containing all nodes in the plan. - * @param jobName The name of the program - */ - public OptimizedPlan( - Collection sources, - Collection sinks, - Collection allNodes, - String jobName, - Plan programPlan) { - this.dataSources = sources; - this.dataSinks = sinks; - this.allNodes = allNodes; - this.jobName = jobName; - this.originalProgram = programPlan; - } - - /** - * Gets the data sources from this OptimizedPlan. - * - * @return The data sources. - */ - public Collection getDataSources() { - return dataSources; - } - - /** - * Gets the data sinks from this OptimizedPlan. - * - * @return The data sinks. - */ - public Collection getDataSinks() { - return dataSinks; - } - - /** - * Gets all the nodes from this OptimizedPlan. - * - * @return All nodes. - */ - public Collection getAllNodes() { - return allNodes; - } - - /** - * Returns the name of the program. - * - * @return The name of the program. - */ - public String getJobName() { - return this.jobName; - } - - /** - * Gets the original program's dataflow plan from which this optimized plan was created. - * - * @return The original program's dataflow plan. - */ - public Plan getOriginalPlan() { - return this.originalProgram; - } - - // ------------------------------------------------------------------------ - - /** - * Applies the given visitor top down to all nodes, starting at the sinks. - * - * @param visitor The visitor to apply to the nodes in this plan. - * @see org.apache.flink.util.Visitable#accept(org.apache.flink.util.Visitor) - */ - @Override - public void accept(Visitor visitor) { - for (SinkPlanNode node : this.dataSinks) { - node.accept(visitor); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java deleted file mode 100644 index fa20201fe177a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java +++ /dev/null @@ -1,605 +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.optimizer.plan; - -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.ResourceSpec; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.OptimizerNode.UnclosedBranchDescriptor; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plandump.DumpableConnection; -import org.apache.flink.optimizer.plandump.DumpableNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Visitable; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * The representation of a data exchange between two operators. The data exchange can realize a - * shipping strategy, which established global properties, and a local strategy, which establishes - * local properties. - * - *

Because we currently deal only with plans where the operator order is fixed, many properties - * are equal among candidates and are determined prior to the enumeration (such as for example - * constant/dynamic path membership). Hence, many methods will delegate to the {@code OptimizerNode} - * that represents the node this candidate was created for. - */ -public abstract class PlanNode implements Visitable, DumpableNode { - - protected final OptimizerNode template; - - protected final List outChannels; - - private List broadcastInputs; - - private final String nodeName; - - private DriverStrategy driverStrategy; // The local strategy (sorting / hashing, ...) - - protected LocalProperties localProps; // local properties of the data produced by this node - - protected GlobalProperties globalProps; // global properties of the data produced by this node - - protected Map - branchPlan; // the actual plan alternative chosen at a branch point - - protected Costs nodeCosts; // the costs incurred by this node - - protected Costs cumulativeCosts; // the cumulative costs of all operators in the sub-tree - - private double - relativeMemoryPerSubTask; // the amount of memory dedicated to each task, in bytes - - private int parallelism; - - private boolean pFlag; // flag for the internal pruning algorithm - - // -------------------------------------------------------------------------------------------- - - public PlanNode(OptimizerNode template, String nodeName, DriverStrategy strategy) { - this.outChannels = new ArrayList(2); - this.broadcastInputs = new ArrayList(); - this.template = template; - this.nodeName = nodeName; - this.driverStrategy = strategy; - - this.parallelism = template.getParallelism(); - - // check, if there is branch at this node. if yes, this candidate must be associated with - // the branching template node. - if (template.isBranching()) { - this.branchPlan = new HashMap(6); - this.branchPlan.put(template, this); - } - } - - protected void mergeBranchPlanMaps(PlanNode pred1, PlanNode pred2) { - mergeBranchPlanMaps(pred1.branchPlan, pred2.branchPlan); - } - - protected void mergeBranchPlanMaps( - Map branchPlan1, Map branchPlan2) { - // merge the branchPlan maps according the template's uncloseBranchesStack - if (this.template.hasUnclosedBranches()) { - if (this.branchPlan == null) { - this.branchPlan = new HashMap(8); - } - - for (UnclosedBranchDescriptor uc : this.template.getOpenBranches()) { - OptimizerNode brancher = uc.getBranchingNode(); - PlanNode selectedCandidate = null; - - if (branchPlan1 != null) { - // predecessor 1 has branching children, see if it got the branch we are looking - // for - selectedCandidate = branchPlan1.get(brancher); - } - - if (selectedCandidate == null && branchPlan2 != null) { - // predecessor 2 has branching children, see if it got the branch we are looking - // for - selectedCandidate = branchPlan2.get(brancher); - } - - // it may be that the branch candidate is only found once the broadcast variables - // are set - if (selectedCandidate != null) { - this.branchPlan.put(brancher, selectedCandidate); - } - } - } - } - - // -------------------------------------------------------------------------------------------- - // Accessors - // -------------------------------------------------------------------------------------------- - - /** - * Gets the node from the optimizer DAG for which this plan candidate node was created. - * - * @return The optimizer's DAG node. - */ - public OptimizerNode getOriginalOptimizerNode() { - return this.template; - } - - /** - * Gets the program operator that this node represents in the plan. - * - * @return The program operator this node represents in the plan. - */ - public Operator getProgramOperator() { - return this.template.getOperator(); - } - - /** - * Gets the name of the plan node. - * - * @return The name of the plan node. - */ - public String getNodeName() { - return this.nodeName; - } - - public int getMemoryConsumerWeight() { - return this.driverStrategy.isMaterializing() ? 1 : 0; - } - - /** - * Gets the memory dedicated to each sub-task for this node. - * - * @return The memory per task, in bytes. - */ - public double getRelativeMemoryPerSubTask() { - return this.relativeMemoryPerSubTask; - } - - /** - * Sets the memory dedicated to each task for this node. - * - * @param relativeMemoryPerSubtask The relative memory per sub-task - */ - public void setRelativeMemoryPerSubtask(double relativeMemoryPerSubtask) { - this.relativeMemoryPerSubTask = relativeMemoryPerSubtask; - } - - /** - * Gets the driver strategy from this node. This determines for example for a match Pact - * whether to use a merge or a hybrid hash strategy. - * - * @return The driver strategy. - */ - public DriverStrategy getDriverStrategy() { - return this.driverStrategy; - } - - /** - * Sets the driver strategy for this node. Usually should not be changed. - * - * @param newDriverStrategy The driver strategy. - */ - public void setDriverStrategy(DriverStrategy newDriverStrategy) { - this.driverStrategy = newDriverStrategy; - } - - public void initProperties(GlobalProperties globals, LocalProperties locals) { - if (this.globalProps != null || this.localProps != null) { - throw new IllegalStateException(); - } - this.globalProps = globals; - this.localProps = locals; - } - - /** - * Gets the local properties from this PlanNode. - * - * @return The local properties. - */ - public LocalProperties getLocalProperties() { - return this.localProps; - } - - /** - * Gets the global properties from this PlanNode. - * - * @return The global properties. - */ - public GlobalProperties getGlobalProperties() { - return this.globalProps; - } - - /** - * Gets the costs incurred by this node. The costs reflect also the costs incurred by the - * shipping strategies of the incoming connections. - * - * @return The node-costs, or null, if not yet set. - */ - public Costs getNodeCosts() { - return this.nodeCosts; - } - - /** - * Gets the cumulative costs of this nose. The cumulative costs are the sum of the costs of this - * node and of all nodes in the subtree below this node. - * - * @return The cumulative costs, or null, if not yet set. - */ - public Costs getCumulativeCosts() { - return this.cumulativeCosts; - } - - public Costs getCumulativeCostsShare() { - if (this.cumulativeCosts == null) { - return null; - } else { - Costs result = cumulativeCosts.clone(); - if (this.template.getOutgoingConnections() != null) { - int outDegree = this.template.getOutgoingConnections().size(); - if (outDegree > 0) { - result.divideBy(outDegree); - } - } - - return result; - } - } - - /** - * Sets the basic cost for this node to the given value, and sets the cumulative costs to those - * costs plus the cost shares of all inputs (regular and broadcast). - * - * @param nodeCosts The already knows costs for this node (this cost a produces by a concrete - * {@code OptimizerNode} subclass. - */ - public void setCosts(Costs nodeCosts) { - // set the node costs - this.nodeCosts = nodeCosts; - - // the cumulative costs are the node costs plus the costs of all inputs - this.cumulativeCosts = nodeCosts.clone(); - - // add all the normal inputs - for (PlanNode pred : getPredecessors()) { - - Costs parentCosts = pred.getCumulativeCostsShare(); - if (parentCosts != null) { - this.cumulativeCosts.addCosts(parentCosts); - } else { - throw new CompilerException( - "Trying to set the costs of an operator before the predecessor costs are computed."); - } - } - - // add all broadcast variable inputs - if (this.broadcastInputs != null) { - for (NamedChannel nc : this.broadcastInputs) { - Costs bcInputCost = nc.getSource().getCumulativeCostsShare(); - if (bcInputCost != null) { - this.cumulativeCosts.addCosts(bcInputCost); - } else { - throw new CompilerException( - "Trying to set the costs of an operator before the broadcast input costs are computed."); - } - } - } - } - - public void setParallelism(int parallelism) { - this.parallelism = parallelism; - } - - public int getParallelism() { - return this.parallelism; - } - - public ResourceSpec getMinResources() { - return this.template.getOperator().getMinResources(); - } - - public ResourceSpec getPreferredResources() { - return this.template.getOperator().getPreferredResources(); - } - - public long getGuaranteedAvailableMemory() { - return this.template.getMinimalMemoryAcrossAllSubTasks(); - } - - public Map getBranchPlan() { - return branchPlan; - } - - // -------------------------------------------------------------------------------------------- - // Input, Predecessors, Successors - // -------------------------------------------------------------------------------------------- - - public abstract Iterable getInputs(); - - @Override - public abstract Iterable getPredecessors(); - - /** Sets a list of all broadcast inputs attached to this node. */ - public void setBroadcastInputs(List broadcastInputs) { - if (broadcastInputs != null) { - this.broadcastInputs = broadcastInputs; - - // update the branch map - for (NamedChannel nc : broadcastInputs) { - PlanNode source = nc.getSource(); - - mergeBranchPlanMaps(branchPlan, source.branchPlan); - } - } - - // do a sanity check that if we are branching, we have now candidates for each branch point - if (this.template.hasUnclosedBranches()) { - if (this.branchPlan == null) { - throw new CompilerException( - "Branching and rejoining logic did not find a candidate for the branching point."); - } - - for (UnclosedBranchDescriptor uc : this.template.getOpenBranches()) { - OptimizerNode brancher = uc.getBranchingNode(); - if (this.branchPlan.get(brancher) == null) { - throw new CompilerException( - "Branching and rejoining logic did not find a candidate for the branching point."); - } - } - } - } - - /** Gets a list of all broadcast inputs attached to this node. */ - public List getBroadcastInputs() { - return this.broadcastInputs; - } - - /** - * Adds a channel to a successor node to this node. - * - * @param channel The channel to the successor. - */ - public void addOutgoingChannel(Channel channel) { - this.outChannels.add(channel); - } - - /** - * Gets a list of all outgoing channels leading to successors. - * - * @return A list of all channels leading to successors. - */ - public List getOutgoingChannels() { - return this.outChannels; - } - - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - - public void updatePropertiesWithUniqueSets(Set

uniqueFieldCombinations) { - if (uniqueFieldCombinations == null || uniqueFieldCombinations.isEmpty()) { - return; - } - for (FieldSet fields : uniqueFieldCombinations) { - this.globalProps.addUniqueFieldCombination(fields); - this.localProps = this.localProps.addUniqueFields(fields); - } - } - - public PlanNode getCandidateAtBranchPoint(OptimizerNode branchPoint) { - if (branchPlan == null) { - return null; - } else { - return this.branchPlan.get(branchPoint); - } - } - - /** Sets the pruning marker to true. */ - public void setPruningMarker() { - this.pFlag = true; - } - - /** - * Checks whether the pruning marker was set. - * - * @return True, if the pruning marker was set, false otherwise. - */ - public boolean isPruneMarkerSet() { - return this.pFlag; - } - - public boolean isOnDynamicPath() { - return this.template.isOnDynamicPath(); - } - - public int getCostWeight() { - return this.template.getCostWeight(); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Checks whether this node has a dam on the way down to the given source node. This method - * returns either that (a) the source node is not found as a (transitive) child of this node, - * (b) the node is found, but no dam is on the path, or (c) the node is found and a dam is on - * the path. - * - * @param source The node on the path to which the dam is sought. - * @return The result whether the node is found and whether a dam is on the path. - */ - public abstract SourceAndDamReport hasDamOnPathDownTo(PlanNode source); - - public FeedbackPropertiesMeetRequirementsReport checkPartialSolutionPropertiesMet( - PlanNode partialSolution, - GlobalProperties feedbackGlobal, - LocalProperties feedbackLocal) { - if (this == partialSolution) { - return FeedbackPropertiesMeetRequirementsReport.PENDING; - } - - boolean found = false; - boolean allMet = true; - boolean allLocallyMet = true; - - for (Channel input : getInputs()) { - FeedbackPropertiesMeetRequirementsReport inputState = - input.getSource() - .checkPartialSolutionPropertiesMet( - partialSolution, feedbackGlobal, feedbackLocal); - - if (inputState == FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION) { - continue; - } else if (inputState == FeedbackPropertiesMeetRequirementsReport.MET) { - found = true; - continue; - } else if (inputState == FeedbackPropertiesMeetRequirementsReport.NOT_MET) { - return FeedbackPropertiesMeetRequirementsReport.NOT_MET; - } else { - found = true; - - // the partial solution was on the path here. check whether the channel requires - // certain properties that are met, or whether the channel introduces new properties - - // if the plan introduces new global properties, then we can stop looking whether - // the feedback properties are sufficient to meet the requirements - if (input.getShipStrategy() != ShipStrategyType.FORWARD - && input.getShipStrategy() != ShipStrategyType.NONE) { - continue; - } - - // first check whether this channel requires something that is not met - if (input.getRequiredGlobalProps() != null - && !input.getRequiredGlobalProps().isMetBy(feedbackGlobal)) { - return FeedbackPropertiesMeetRequirementsReport.NOT_MET; - } - - // in general, not everything is met here already - allMet = false; - - // if the plan introduces new local properties, we can stop checking for matching - // local properties - if (inputState != FeedbackPropertiesMeetRequirementsReport.PENDING_LOCAL_MET) { - - if (input.getLocalStrategy() == LocalStrategy.NONE) { - - if (input.getRequiredLocalProps() != null - && !input.getRequiredLocalProps().isMetBy(feedbackLocal)) { - return FeedbackPropertiesMeetRequirementsReport.NOT_MET; - } - - allLocallyMet = false; - } - } - } - } - - if (!found) { - return FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION; - } else if (allMet) { - return FeedbackPropertiesMeetRequirementsReport.MET; - } else if (allLocallyMet) { - return FeedbackPropertiesMeetRequirementsReport.PENDING_LOCAL_MET; - } else { - return FeedbackPropertiesMeetRequirementsReport.PENDING; - } - } - - // -------------------------------------------------------------------------------------------- - - @Override - public String toString() { - return this.template.getOperatorName() - + " \"" - + getProgramOperator().getName() - + "\" : " - + this.driverStrategy - + " [[ " - + this.globalProps - + " ]] [[ " - + this.localProps - + " ]]"; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public OptimizerNode getOptimizerNode() { - return this.template; - } - - @Override - public PlanNode getPlanNode() { - return this; - } - - @Override - public Iterable> getDumpableInputs() { - List> allInputs = - new ArrayList>(); - - for (Channel c : getInputs()) { - allInputs.add(c); - } - - for (NamedChannel c : getBroadcastInputs()) { - allInputs.add(c); - } - - return allInputs; - } - - // -------------------------------------------------------------------------------------------- - - public static enum SourceAndDamReport { - NOT_FOUND, - FOUND_SOURCE, - FOUND_SOURCE_AND_DAM - } - - public static enum FeedbackPropertiesMeetRequirementsReport { - /** Indicates that the path is irrelevant */ - NO_PARTIAL_SOLUTION, - - /** - * Indicates that the question whether the properties are met has been determined pending - * dependent on global and local properties - */ - PENDING, - - /** - * Indicates that the question whether the properties are met has been determined pending - * dependent on global properties only - */ - PENDING_LOCAL_MET, - - /** Indicates that the question whether the properties are met has been determined true */ - MET, - - /** Indicates that the question whether the properties are met has been determined false */ - NOT_MET - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SingleInputPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SingleInputPlanNode.java deleted file mode 100644 index 605bd5d05c7bc..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SingleInputPlanNode.java +++ /dev/null @@ -1,279 +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.optimizer.plan; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.runtime.operators.DamBehavior; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** */ -public class SingleInputPlanNode extends PlanNode { - - protected final Channel input; - - protected final FieldList[] driverKeys; - - protected final boolean[][] driverSortOrders; - - private TypeComparatorFactory[] comparators; - - public Object postPassHelper; - - // -------------------------------------------------------------------------------------------- - - public SingleInputPlanNode( - OptimizerNode template, String nodeName, Channel input, DriverStrategy driverStrategy) { - this(template, nodeName, input, driverStrategy, null, null); - } - - public SingleInputPlanNode( - OptimizerNode template, - String nodeName, - Channel input, - DriverStrategy driverStrategy, - FieldList driverKeyFields) { - this( - template, - nodeName, - input, - driverStrategy, - driverKeyFields, - getTrueArray(driverKeyFields.size())); - } - - public SingleInputPlanNode( - OptimizerNode template, - String nodeName, - Channel input, - DriverStrategy driverStrategy, - FieldList driverKeyFields, - boolean[] driverSortOrders) { - super(template, nodeName, driverStrategy); - this.input = input; - - this.comparators = new TypeComparatorFactory[driverStrategy.getNumRequiredComparators()]; - this.driverKeys = new FieldList[driverStrategy.getNumRequiredComparators()]; - this.driverSortOrders = new boolean[driverStrategy.getNumRequiredComparators()][]; - - if (driverStrategy.getNumRequiredComparators() > 0) { - this.driverKeys[0] = driverKeyFields; - this.driverSortOrders[0] = driverSortOrders; - } - - if (this.input.getShipStrategy() == ShipStrategyType.BROADCAST) { - this.input.setReplicationFactor(getParallelism()); - } - - final PlanNode predNode = input.getSource(); - - if (predNode.branchPlan != null && !predNode.branchPlan.isEmpty()) { - - if (this.branchPlan == null) { - this.branchPlan = new HashMap(); - } - this.branchPlan.putAll(predNode.branchPlan); - } - } - - // -------------------------------------------------------------------------------------------- - - public SingleInputNode getSingleInputNode() { - if (this.template instanceof SingleInputNode) { - return (SingleInputNode) this.template; - } else { - throw new RuntimeException(); - } - } - - /** - * Gets the input channel to this node. - * - * @return The input channel to this node. - */ - public Channel getInput() { - return this.input; - } - - /** - * Gets the predecessor of this node, i.e. the source of the input channel. - * - * @return The predecessor of this node. - */ - public PlanNode getPredecessor() { - return this.input.getSource(); - } - - /** - * Sets the key field indexes for the specified driver comparator. - * - * @param keys The key field indexes for the specified driver comparator. - * @param id The ID of the driver comparator. - */ - public void setDriverKeyInfo(FieldList keys, int id) { - this.setDriverKeyInfo(keys, getTrueArray(keys.size()), id); - } - - /** - * Sets the key field information for the specified driver comparator. - * - * @param keys The key field indexes for the specified driver comparator. - * @param sortOrder The key sort order for the specified driver comparator. - * @param id The ID of the driver comparator. - */ - public void setDriverKeyInfo(FieldList keys, boolean[] sortOrder, int id) { - if (id < 0 || id >= driverKeys.length) { - throw new CompilerException( - "Invalid id for driver key information. DriverStrategy requires only " - + super.getDriverStrategy().getNumRequiredComparators() - + " comparators."); - } - this.driverKeys[id] = keys; - this.driverSortOrders[id] = sortOrder; - } - - /** - * Gets the key field indexes for the specified driver comparator. - * - * @param id The id of the driver comparator for which the key field indexes are requested. - * @return The key field indexes of the specified driver comparator. - */ - public FieldList getKeys(int id) { - return this.driverKeys[id]; - } - - /** - * Gets the sort order for the specified driver comparator. - * - * @param id The id of the driver comparator for which the sort order is requested. - * @return The sort order of the specified driver comparator. - */ - public boolean[] getSortOrders(int id) { - return driverSortOrders[id]; - } - - /** - * Gets the specified comparator from this PlanNode. - * - * @param id The ID of the requested comparator. - * @return The specified comparator. - */ - public TypeComparatorFactory getComparator(int id) { - return comparators[id]; - } - - /** - * Sets the specified comparator for this PlanNode. - * - * @param comparator The comparator to set. - * @param id The ID of the comparator to set. - */ - public void setComparator(TypeComparatorFactory comparator, int id) { - this.comparators[id] = comparator; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - this.input.getSource().accept(visitor); - - for (Channel broadcastInput : getBroadcastInputs()) { - broadcastInput.getSource().accept(visitor); - } - - visitor.postVisit(this); - } - } - - @Override - public Iterable getPredecessors() { - if (getBroadcastInputs() == null || getBroadcastInputs().isEmpty()) { - return Collections.singleton(this.input.getSource()); - } else { - List preds = new ArrayList(); - preds.add(input.getSource()); - - for (Channel c : getBroadcastInputs()) { - preds.add(c.getSource()); - } - - return preds; - } - } - - @Override - public Iterable getInputs() { - return Collections.singleton(this.input); - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } - SourceAndDamReport res = this.input.getSource().hasDamOnPathDownTo(source); - if (res == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (res == FOUND_SOURCE) { - return (this.input.getLocalStrategy().dams() - || this.input.getTempMode().breaksPipeline() - || getDriverStrategy().firstDam() == DamBehavior.FULL_DAM) - ? FOUND_SOURCE_AND_DAM - : FOUND_SOURCE; - } else { - // NOT_FOUND - // check the broadcast inputs - - for (NamedChannel nc : getBroadcastInputs()) { - SourceAndDamReport bcRes = nc.getSource().hasDamOnPathDownTo(source); - if (bcRes != NOT_FOUND) { - // broadcast inputs are always dams - return FOUND_SOURCE_AND_DAM; - } - } - return NOT_FOUND; - } - } - - // -------------------------------------------------------------------------------------------- - - protected static boolean[] getTrueArray(int length) { - final boolean[] a = new boolean[length]; - for (int i = 0; i < length; i++) { - a[i] = true; - } - return a; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java deleted file mode 100644 index 85c8d3a9fc92e..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkJoinerPlanNode.java +++ /dev/null @@ -1,73 +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.optimizer.plan; - -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.SinkJoiner; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.List; - -/** */ -public class SinkJoinerPlanNode extends DualInputPlanNode { - - public SinkJoinerPlanNode(SinkJoiner template, Channel input1, Channel input2) { - super(template, "", input1, input2, DriverStrategy.BINARY_NO_OP); - } - - // -------------------------------------------------------------------------------------------- - - public void setCosts(Costs nodeCosts) { - // the plan enumeration logic works as for regular two-input-operators, which is important - // because of the branch handling logic. it does pick redistributing network channels - // between the sink and the sink joiner, because sinks joiner has a different parallelism - // than the sink. - // we discard any cost and simply use the sum of the costs from the two children. - - Costs totalCosts = getInput1().getSource().getCumulativeCosts().clone(); - totalCosts.addCosts(getInput2().getSource().getCumulativeCosts()); - super.setCosts(totalCosts); - } - - // -------------------------------------------------------------------------------------------- - - public void getDataSinks(List sinks) { - final PlanNode in1 = this.input1.getSource(); - final PlanNode in2 = this.input2.getSource(); - - if (in1 instanceof SinkPlanNode) { - sinks.add((SinkPlanNode) in1); - } else if (in1 instanceof SinkJoinerPlanNode) { - ((SinkJoinerPlanNode) in1).getDataSinks(sinks); - } else { - throw new CompilerException( - "Illegal child node for a sink joiner utility node: Neither Sink nor Sink Joiner"); - } - - if (in2 instanceof SinkPlanNode) { - sinks.add((SinkPlanNode) in2); - } else if (in2 instanceof SinkJoinerPlanNode) { - ((SinkJoinerPlanNode) in2).getDataSinks(sinks); - } else { - throw new CompilerException( - "Illegal child node for a sink joiner utility node: Neither Sink nor Sink Joiner"); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkPlanNode.java deleted file mode 100644 index 6a3a555c36ca4..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SinkPlanNode.java +++ /dev/null @@ -1,47 +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.optimizer.plan; - -import org.apache.flink.optimizer.dag.DataSinkNode; -import org.apache.flink.runtime.operators.DriverStrategy; - -/** Plan candidate node for data flow sinks. */ -public class SinkPlanNode extends SingleInputPlanNode { - - /** - * Constructs a new sink candidate node that uses NONE as its local strategy. Note that - * local sorting and range partitioning are handled by the incoming channel already. - * - * @param template The template optimizer node that this candidate is created for. - */ - public SinkPlanNode(DataSinkNode template, String nodeName, Channel input) { - super(template, nodeName, input, DriverStrategy.NONE); - - this.globalProps = input.getGlobalProperties().clone(); - this.localProps = input.getLocalProperties().clone(); - } - - public DataSinkNode getSinkNode() { - if (this.template instanceof DataSinkNode) { - return (DataSinkNode) this.template; - } else { - throw new RuntimeException(); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SolutionSetPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SolutionSetPlanNode.java deleted file mode 100644 index 4378f3a5b6f02..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SolutionSetPlanNode.java +++ /dev/null @@ -1,119 +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.optimizer.plan; - -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.SolutionSetNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.Collections; -import java.util.HashMap; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** Plan candidate node for partial solution of a bulk iteration. */ -public class SolutionSetPlanNode extends PlanNode { - - private static final Costs NO_COSTS = new Costs(); - - private WorksetIterationPlanNode containingIterationNode; - - private final Channel initialInput; - - public Object postPassHelper; - - public SolutionSetPlanNode( - SolutionSetNode template, - String nodeName, - GlobalProperties gProps, - LocalProperties lProps, - Channel initialInput) { - super(template, nodeName, DriverStrategy.NONE); - - this.globalProps = gProps; - this.localProps = lProps; - this.initialInput = initialInput; - - // the node incurs no cost - this.nodeCosts = NO_COSTS; - this.cumulativeCosts = NO_COSTS; - - if (initialInput.getSource().branchPlan != null - && initialInput.getSource().branchPlan.size() > 0) { - if (this.branchPlan == null) { - this.branchPlan = new HashMap(); - } - - this.branchPlan.putAll(initialInput.getSource().branchPlan); - } - } - - // -------------------------------------------------------------------------------------------- - - public SolutionSetNode getSolutionSetNode() { - return (SolutionSetNode) this.template; - } - - public WorksetIterationPlanNode getContainingIterationNode() { - return this.containingIterationNode; - } - - public void setContainingIterationNode(WorksetIterationPlanNode containingIterationNode) { - this.containingIterationNode = containingIterationNode; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - visitor.postVisit(this); - } - } - - @Override - public Iterable getPredecessors() { - return Collections.emptyList(); - } - - @Override - public Iterable getInputs() { - return Collections.emptyList(); - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE_AND_DAM; - } - - SourceAndDamReport res = this.initialInput.getSource().hasDamOnPathDownTo(source); - if (res == FOUND_SOURCE_AND_DAM || res == FOUND_SOURCE) { - return FOUND_SOURCE_AND_DAM; - } else { - return NOT_FOUND; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SourcePlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SourcePlanNode.java deleted file mode 100644 index 12573bd7949d8..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/SourcePlanNode.java +++ /dev/null @@ -1,110 +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.optimizer.plan; - -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.optimizer.dag.DataSourceNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.Collections; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** Plan candidate node for data flow sources that have no input and no special strategies. */ -public class SourcePlanNode extends PlanNode { - - private TypeSerializerFactory serializer; - - /** - * Constructs a new source candidate node that uses NONE as its local strategy. - * - * @param template The template optimizer node that this candidate is created for. - */ - public SourcePlanNode(DataSourceNode template, String nodeName) { - this(template, nodeName, new GlobalProperties(), new LocalProperties()); - } - - public SourcePlanNode( - DataSourceNode template, - String nodeName, - GlobalProperties gprops, - LocalProperties lprops) { - super(template, nodeName, DriverStrategy.NONE); - - this.globalProps = gprops; - this.localProps = lprops; - updatePropertiesWithUniqueSets(template.getUniqueFields()); - } - - // -------------------------------------------------------------------------------------------- - - public DataSourceNode getDataSourceNode() { - return (DataSourceNode) this.template; - } - - /** - * Gets the serializer from this PlanNode. - * - * @return The serializer. - */ - public TypeSerializerFactory getSerializer() { - return serializer; - } - - /** - * Sets the serializer for this PlanNode. - * - * @param serializer The serializer to set. - */ - public void setSerializer(TypeSerializerFactory serializer) { - this.serializer = serializer; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - visitor.postVisit(this); - } - } - - @Override - public Iterable getPredecessors() { - return Collections.emptyList(); - } - - @Override - public Iterable getInputs() { - return Collections.emptyList(); - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } else { - return NOT_FOUND; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/WorksetIterationPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/WorksetIterationPlanNode.java deleted file mode 100644 index 82cc2d2d41e65..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/WorksetIterationPlanNode.java +++ /dev/null @@ -1,265 +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.optimizer.plan; - -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.WorksetIterationNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; - -/** A node in the execution, representing a workset iteration (delta iteration). */ -public class WorksetIterationPlanNode extends DualInputPlanNode implements IterationPlanNode { - - private final SolutionSetPlanNode solutionSetPlanNode; - - private final WorksetPlanNode worksetPlanNode; - - private final PlanNode solutionSetDeltaPlanNode; - - private final PlanNode nextWorkSetPlanNode; - - private TypeSerializerFactory worksetSerializer; - - private TypeSerializerFactory solutionSetSerializer; - - private TypeComparatorFactory solutionSetComparator; - - private boolean immediateSolutionSetUpdate; - - public Object postPassHelper; - - private TypeSerializerFactory serializerForIterationChannel; - - // -------------------------------------------------------------------------------------------- - - public WorksetIterationPlanNode( - WorksetIterationNode template, - String nodeName, - Channel initialSolutionSet, - Channel initialWorkset, - SolutionSetPlanNode solutionSetPlanNode, - WorksetPlanNode worksetPlanNode, - PlanNode nextWorkSetPlanNode, - PlanNode solutionSetDeltaPlanNode) { - super(template, nodeName, initialSolutionSet, initialWorkset, DriverStrategy.BINARY_NO_OP); - this.solutionSetPlanNode = solutionSetPlanNode; - this.worksetPlanNode = worksetPlanNode; - this.solutionSetDeltaPlanNode = solutionSetDeltaPlanNode; - this.nextWorkSetPlanNode = nextWorkSetPlanNode; - - mergeBranchPlanMaps(); - } - - // -------------------------------------------------------------------------------------------- - - public WorksetIterationNode getIterationNode() { - if (this.template instanceof WorksetIterationNode) { - return (WorksetIterationNode) this.template; - } else { - throw new RuntimeException(); - } - } - - public SolutionSetPlanNode getSolutionSetPlanNode() { - return this.solutionSetPlanNode; - } - - public WorksetPlanNode getWorksetPlanNode() { - return this.worksetPlanNode; - } - - public PlanNode getSolutionSetDeltaPlanNode() { - return this.solutionSetDeltaPlanNode; - } - - public PlanNode getNextWorkSetPlanNode() { - return this.nextWorkSetPlanNode; - } - - public Channel getInitialSolutionSetInput() { - return getInput1(); - } - - public Channel getInitialWorksetInput() { - return getInput2(); - } - - public void setImmediateSolutionSetUpdate(boolean immediateUpdate) { - this.immediateSolutionSetUpdate = immediateUpdate; - } - - public boolean isImmediateSolutionSetUpdate() { - return this.immediateSolutionSetUpdate; - } - - public FieldList getSolutionSetKeyFields() { - return getIterationNode().getSolutionSetKeyFields(); - } - - // -------------------------------------------------------------------------------------------- - - public TypeSerializerFactory getWorksetSerializer() { - return worksetSerializer; - } - - public void setWorksetSerializer(TypeSerializerFactory worksetSerializer) { - this.worksetSerializer = worksetSerializer; - } - - public TypeSerializerFactory getSolutionSetSerializer() { - return solutionSetSerializer; - } - - public void setSolutionSetSerializer(TypeSerializerFactory solutionSetSerializer) { - this.solutionSetSerializer = solutionSetSerializer; - } - - public TypeComparatorFactory getSolutionSetComparator() { - return solutionSetComparator; - } - - public void setSolutionSetComparator(TypeComparatorFactory solutionSetComparator) { - this.solutionSetComparator = solutionSetComparator; - } - - // -------------------------------------------------------------------------------------------- - - public void setCosts(Costs nodeCosts) { - // add the costs from the step function - nodeCosts.addCosts(this.solutionSetDeltaPlanNode.getCumulativeCostsShare()); - nodeCosts.addCosts(this.nextWorkSetPlanNode.getCumulativeCostsShare()); - - super.setCosts(nodeCosts); - } - - public int getMemoryConsumerWeight() { - // solution set index and workset back channel - return 2; - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } - - SourceAndDamReport fromOutside = super.hasDamOnPathDownTo(source); - - if (fromOutside == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (fromOutside == FOUND_SOURCE) { - // we always have a dam in the solution set index - return FOUND_SOURCE_AND_DAM; - } else { - SourceAndDamReport fromNextWorkset = nextWorkSetPlanNode.hasDamOnPathDownTo(source); - - if (fromNextWorkset == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (fromNextWorkset == FOUND_SOURCE) { - return FOUND_SOURCE_AND_DAM; - } else { - return this.solutionSetDeltaPlanNode.hasDamOnPathDownTo(source); - } - } - } - - @Override - public void acceptForStepFunction(Visitor visitor) { - this.solutionSetDeltaPlanNode.accept(visitor); - this.nextWorkSetPlanNode.accept(visitor); - } - - /** - * Merging can only take place after the solutionSetDelta and nextWorkset PlanNode has been set, - * because they can contain also some of the branching nodes. - */ - @Override - protected void mergeBranchPlanMaps( - Map branchPlan1, Map branchPlan2) {} - - protected void mergeBranchPlanMaps() { - Map branchPlan1 = input1.getSource().branchPlan; - Map branchPlan2 = input2.getSource().branchPlan; - - // merge the branchPlan maps according the template's uncloseBranchesStack - if (this.template.hasUnclosedBranches()) { - if (this.branchPlan == null) { - this.branchPlan = new HashMap(8); - } - - for (OptimizerNode.UnclosedBranchDescriptor uc : this.template.getOpenBranches()) { - OptimizerNode brancher = uc.getBranchingNode(); - PlanNode selectedCandidate = null; - - if (branchPlan1 != null) { - // predecessor 1 has branching children, see if it got the branch we are looking - // for - selectedCandidate = branchPlan1.get(brancher); - } - - if (selectedCandidate == null && branchPlan2 != null) { - // predecessor 2 has branching children, see if it got the branch we are looking - // for - selectedCandidate = branchPlan2.get(brancher); - } - - if (selectedCandidate == null - && getSolutionSetDeltaPlanNode() != null - && getSolutionSetDeltaPlanNode().branchPlan != null) { - selectedCandidate = getSolutionSetDeltaPlanNode().branchPlan.get(brancher); - } - - if (selectedCandidate == null - && getNextWorkSetPlanNode() != null - && getNextWorkSetPlanNode().branchPlan != null) { - selectedCandidate = getNextWorkSetPlanNode().branchPlan.get(brancher); - } - - if (selectedCandidate == null) { - throw new CompilerException( - "Candidates for a node with open branches are missing information about the selected candidate "); - } - - this.branchPlan.put(brancher, selectedCandidate); - } - } - } - - // -------------------------------------------------------------------------------------------- - - public TypeSerializerFactory getSerializerForIterationChannel() { - return serializerForIterationChannel; - } - - public void setSerializerForIterationChannel( - TypeSerializerFactory serializerForIterationChannel) { - this.serializerForIterationChannel = serializerForIterationChannel; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/WorksetPlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/WorksetPlanNode.java deleted file mode 100644 index 3aef1919816f3..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/WorksetPlanNode.java +++ /dev/null @@ -1,125 +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.optimizer.plan; - -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.WorksetNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.runtime.operators.DamBehavior; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.Collections; -import java.util.HashMap; - -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.FOUND_SOURCE_AND_DAM; -import static org.apache.flink.optimizer.plan.PlanNode.SourceAndDamReport.NOT_FOUND; - -/** Plan candidate node for partial solution of a bulk iteration. */ -public class WorksetPlanNode extends PlanNode { - - private static final Costs NO_COSTS = new Costs(); - - private WorksetIterationPlanNode containingIterationNode; - - private final Channel initialInput; - - public Object postPassHelper; - - public WorksetPlanNode( - WorksetNode template, - String nodeName, - GlobalProperties gProps, - LocalProperties lProps, - Channel initialInput) { - super(template, nodeName, DriverStrategy.NONE); - - this.globalProps = gProps; - this.localProps = lProps; - this.initialInput = initialInput; - - // the node incurs no cost - this.nodeCosts = NO_COSTS; - this.cumulativeCosts = NO_COSTS; - - if (initialInput.getSource().branchPlan != null - && initialInput.getSource().branchPlan.size() > 0) { - if (this.branchPlan == null) { - this.branchPlan = new HashMap(); - } - - this.branchPlan.putAll(initialInput.getSource().branchPlan); - } - } - - // -------------------------------------------------------------------------------------------- - - public WorksetNode getWorksetNode() { - return (WorksetNode) this.template; - } - - public WorksetIterationPlanNode getContainingIterationNode() { - return this.containingIterationNode; - } - - public void setContainingIterationNode(WorksetIterationPlanNode containingIterationNode) { - this.containingIterationNode = containingIterationNode; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public void accept(Visitor visitor) { - if (visitor.preVisit(this)) { - visitor.postVisit(this); - } - } - - @Override - public Iterable getPredecessors() { - return Collections.emptyList(); - } - - @Override - public Iterable getInputs() { - return Collections.emptyList(); - } - - @Override - public SourceAndDamReport hasDamOnPathDownTo(PlanNode source) { - if (source == this) { - return FOUND_SOURCE; - } - SourceAndDamReport res = this.initialInput.getSource().hasDamOnPathDownTo(source); - if (res == FOUND_SOURCE_AND_DAM) { - return FOUND_SOURCE_AND_DAM; - } else if (res == FOUND_SOURCE) { - return (this.initialInput.getLocalStrategy().dams() - || this.initialInput.getTempMode().breaksPipeline() - || getDriverStrategy().firstDam() == DamBehavior.FULL_DAM) - ? FOUND_SOURCE_AND_DAM - : FOUND_SOURCE; - } else { - return NOT_FOUND; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/DumpableConnection.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/DumpableConnection.java deleted file mode 100644 index 31da0a8d33c4a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/DumpableConnection.java +++ /dev/null @@ -1,29 +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.optimizer.plandump; - -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -/** */ -public interface DumpableConnection> { - - public DumpableNode getSource(); - - public ShipStrategyType getShipStrategy(); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/DumpableNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/DumpableNode.java deleted file mode 100644 index 389708bd57ef2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/DumpableNode.java +++ /dev/null @@ -1,39 +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.optimizer.plandump; - -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.plan.PlanNode; - -/** */ -public interface DumpableNode> { - - /** - * Gets an iterator over the predecessors. - * - * @return An iterator over the predecessors. - */ - Iterable getPredecessors(); - - Iterable> getDumpableInputs(); - - OptimizerNode getOptimizerNode(); - - PlanNode getPlanNode(); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/ExecutionPlanJSONGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/ExecutionPlanJSONGenerator.java deleted file mode 100644 index 389c04cb8efe2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/ExecutionPlanJSONGenerator.java +++ /dev/null @@ -1,49 +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.optimizer.plandump; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.ExecutionPlanUtil; -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; - -/** - * Utility for extracting an execution plan (as JSON) from a given {@link Plan}. - * - *

We need this util here in the optimizer because it is the only module that has {@link - * Optimizer}, {@link OptimizedPlan}, and {@link PlanJSONDumpGenerator} available. We use this - * reflectively from the batch execution environments to generate the plan, which we cannot do - * there. It is used from {@link ExecutionPlanUtil}. - */ -@SuppressWarnings("unused") -public class ExecutionPlanJSONGenerator implements ExecutionPlanUtil.ExecutionPlanJSONGenerator { - - @Override - public String getExecutionPlan(Plan plan) { - Optimizer opt = - new Optimizer( - new DataStatistics(), new DefaultCostEstimator(), new Configuration()); - OptimizedPlan optPlan = opt.compile(plan); - return new PlanJSONDumpGenerator().getOptimizerPlanAsJSON(optPlan); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java deleted file mode 100644 index 491f1d98201e4..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java +++ /dev/null @@ -1,737 +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.optimizer.plandump; - -import org.apache.flink.api.common.operators.CompilerHints; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.BinaryUnionNode; -import org.apache.flink.optimizer.dag.BulkIterationNode; -import org.apache.flink.optimizer.dag.DataSinkNode; -import org.apache.flink.optimizer.dag.DataSourceNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.dag.WorksetIterationNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.StringUtils; - -import org.apache.commons.text.StringEscapeUtils; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; -import java.util.Map; - -public class PlanJSONDumpGenerator { - - private Map, Integer> nodeIds; // resolves pact nodes to ids - - private int nodeCnt; - - private boolean encodeForHTML; - - // -------------------------------------------------------------------------------------------- - - public void setEncodeForHTML(boolean encodeForHTML) { - this.encodeForHTML = encodeForHTML; - } - - public boolean isEncodeForHTML() { - return encodeForHTML; - } - - public void dumpPactPlanAsJSON(List nodes, PrintWriter writer) { - @SuppressWarnings("unchecked") - List> n = (List>) (List) nodes; - compilePlanToJSON(n, writer); - } - - public String getPactPlanAsJSON(List nodes) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - dumpPactPlanAsJSON(nodes, pw); - return sw.toString(); - } - - public void dumpOptimizerPlanAsJSON(OptimizedPlan plan, File toFile) throws IOException { - PrintWriter pw = null; - try { - pw = new PrintWriter(new FileOutputStream(toFile), false); - dumpOptimizerPlanAsJSON(plan, pw); - pw.flush(); - } finally { - if (pw != null) { - pw.close(); - } - } - } - - public String getOptimizerPlanAsJSON(OptimizedPlan plan) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - dumpOptimizerPlanAsJSON(plan, pw); - pw.close(); - return sw.toString(); - } - - public void dumpOptimizerPlanAsJSON(OptimizedPlan plan, PrintWriter writer) { - Collection sinks = plan.getDataSinks(); - if (sinks instanceof List) { - dumpOptimizerPlanAsJSON((List) sinks, writer); - } else { - List n = new ArrayList(); - n.addAll(sinks); - dumpOptimizerPlanAsJSON(n, writer); - } - } - - public void dumpOptimizerPlanAsJSON(List nodes, PrintWriter writer) { - @SuppressWarnings("unchecked") - List> n = (List>) (List) nodes; - compilePlanToJSON(n, writer); - } - - // -------------------------------------------------------------------------------------------- - - private void compilePlanToJSON(List> nodes, PrintWriter writer) { - // initialization to assign node ids - this.nodeIds = new HashMap, Integer>(); - this.nodeCnt = 0; - - // JSON header - writer.print("{\n\t\"nodes\": [\n\n"); - - // Generate JSON for plan - for (int i = 0; i < nodes.size(); i++) { - visit(nodes.get(i), writer, i == 0); - } - - // JSON Footer - writer.println("\n\t]\n}"); - } - - private boolean visit(DumpableNode node, PrintWriter writer, boolean first) { - // check for duplicate traversal - if (this.nodeIds.containsKey(node)) { - return false; - } - - // assign an id first - this.nodeIds.put(node, this.nodeCnt++); - - // then recurse - for (DumpableNode child : node.getPredecessors()) { - // This is important, because when the node was already in the graph it is not allowed - // to set first to false! - if (visit(child, writer, first)) { - first = false; - } - } - - // check if this node should be skipped from the dump - final OptimizerNode n = node.getOptimizerNode(); - - // ------------------ dump after the ascend --------------------- - // start a new node and output node id - if (!first) { - writer.print(",\n"); - } - // open the node - writer.print("\t{\n"); - - // recurse, it is is an iteration node - if (node instanceof BulkIterationNode || node instanceof BulkIterationPlanNode) { - - DumpableNode innerChild = - node instanceof BulkIterationNode - ? ((BulkIterationNode) node).getNextPartialSolution() - : ((BulkIterationPlanNode) node).getRootOfStepFunction(); - - DumpableNode begin = - node instanceof BulkIterationNode - ? ((BulkIterationNode) node).getPartialSolution() - : ((BulkIterationPlanNode) node).getPartialSolutionPlanNode(); - - writer.print("\t\t\"step_function\": [\n"); - - visit(innerChild, writer, true); - - writer.print("\n\t\t],\n"); - writer.print("\t\t\"partial_solution\": " + this.nodeIds.get(begin) + ",\n"); - writer.print("\t\t\"next_partial_solution\": " + this.nodeIds.get(innerChild) + ",\n"); - } else if (node instanceof WorksetIterationNode - || node instanceof WorksetIterationPlanNode) { - - DumpableNode worksetRoot = - node instanceof WorksetIterationNode - ? ((WorksetIterationNode) node).getNextWorkset() - : ((WorksetIterationPlanNode) node).getNextWorkSetPlanNode(); - DumpableNode solutionDelta = - node instanceof WorksetIterationNode - ? ((WorksetIterationNode) node).getSolutionSetDelta() - : ((WorksetIterationPlanNode) node).getSolutionSetDeltaPlanNode(); - - DumpableNode workset = - node instanceof WorksetIterationNode - ? ((WorksetIterationNode) node).getWorksetNode() - : ((WorksetIterationPlanNode) node).getWorksetPlanNode(); - DumpableNode solutionSet = - node instanceof WorksetIterationNode - ? ((WorksetIterationNode) node).getSolutionSetNode() - : ((WorksetIterationPlanNode) node).getSolutionSetPlanNode(); - - writer.print("\t\t\"step_function\": [\n"); - - visit(worksetRoot, writer, true); - visit(solutionDelta, writer, false); - - writer.print("\n\t\t],\n"); - writer.print("\t\t\"workset\": " + this.nodeIds.get(workset) + ",\n"); - writer.print("\t\t\"solution_set\": " + this.nodeIds.get(solutionSet) + ",\n"); - writer.print("\t\t\"next_workset\": " + this.nodeIds.get(worksetRoot) + ",\n"); - writer.print("\t\t\"solution_delta\": " + this.nodeIds.get(solutionDelta) + ",\n"); - } - - // print the id - writer.print("\t\t\"id\": " + this.nodeIds.get(node)); - - final String type; - String contents; - if (n instanceof DataSinkNode) { - type = "sink"; - contents = n.getOperator().toString(); - } else if (n instanceof DataSourceNode) { - type = "source"; - contents = n.getOperator().toString(); - } else if (n instanceof BulkIterationNode) { - type = "bulk_iteration"; - contents = n.getOperator().getName(); - } else if (n instanceof WorksetIterationNode) { - type = "workset_iteration"; - contents = n.getOperator().getName(); - } else if (n instanceof BinaryUnionNode) { - type = "pact"; - contents = ""; - } else { - type = "pact"; - contents = n.getOperator().getName(); - } - - contents = StringUtils.showControlCharacters(contents); - if (encodeForHTML) { - contents = StringEscapeUtils.escapeHtml4(contents); - contents = contents.replace("\\", "\"); - } - - String name = n.getOperatorName(); - if (name.equals("Reduce") - && (node instanceof SingleInputPlanNode) - && ((SingleInputPlanNode) node).getDriverStrategy() - == DriverStrategy.SORTED_GROUP_COMBINE) { - name = "Combine"; - } - - // output the type identifier - writer.print(",\n\t\t\"type\": \"" + type + "\""); - - // output node name - writer.print(",\n\t\t\"pact\": \"" + name + "\""); - - // output node contents - writer.print(",\n\t\t\"contents\": \"" + contents + "\""); - - // parallelism - writer.print( - ",\n\t\t\"parallelism\": \"" - + (n.getParallelism() >= 1 ? n.getParallelism() : "default") - + "\""); - - // output node predecessors - Iterator> inConns = node.getDumpableInputs().iterator(); - String child1name = "", child2name = ""; - - if (inConns != null && inConns.hasNext()) { - // start predecessor list - writer.print(",\n\t\t\"predecessors\": ["); - int inputNum = 0; - - while (inConns.hasNext()) { - final DumpableConnection inConn = inConns.next(); - final DumpableNode source = inConn.getSource(); - writer.print(inputNum == 0 ? "\n" : ",\n"); - if (inputNum == 0) { - child1name += child1name.length() > 0 ? ", " : ""; - child1name += - source.getOptimizerNode().getOperator().getName() - + " (id: " - + this.nodeIds.get(source) - + ")"; - } else if (inputNum == 1) { - child2name += child2name.length() > 0 ? ", " : ""; - child2name += - source.getOptimizerNode().getOperator().getName() - + " (id: " - + this.nodeIds.get(source) - + ")"; - } - - // output predecessor id - writer.print("\t\t\t{\"id\": " + this.nodeIds.get(source)); - - // output connection side - if (inConns.hasNext() || inputNum > 0) { - writer.print(", \"side\": \"" + (inputNum == 0 ? "first" : "second") + "\""); - } - // output shipping strategy and channel type - final Channel channel = (inConn instanceof Channel) ? (Channel) inConn : null; - final ShipStrategyType shipType = - channel != null ? channel.getShipStrategy() : inConn.getShipStrategy(); - - String shipStrategy = null; - if (shipType != null) { - switch (shipType) { - case NONE: - // nothing - break; - case FORWARD: - shipStrategy = "Forward"; - break; - case BROADCAST: - shipStrategy = "Broadcast"; - break; - case PARTITION_HASH: - shipStrategy = "Hash Partition"; - break; - case PARTITION_RANGE: - shipStrategy = "Range Partition"; - break; - case PARTITION_RANDOM: - shipStrategy = "Redistribute"; - break; - case PARTITION_FORCED_REBALANCE: - shipStrategy = "Rebalance"; - break; - case PARTITION_CUSTOM: - shipStrategy = "Custom Partition"; - break; - default: - throw new CompilerException( - "Unknown ship strategy '" - + inConn.getShipStrategy().name() - + "' in JSON generator."); - } - } - - if (channel != null - && channel.getShipStrategyKeys() != null - && channel.getShipStrategyKeys().size() > 0) { - shipStrategy += - " on " - + (channel.getShipStrategySortOrder() == null - ? channel.getShipStrategyKeys().toString() - : Utils.createOrdering( - channel.getShipStrategyKeys(), - channel.getShipStrategySortOrder()) - .toString()); - } - - if (shipStrategy != null) { - writer.print(", \"ship_strategy\": \"" + shipStrategy + "\""); - } - - if (channel != null) { - String localStrategy = null; - switch (channel.getLocalStrategy()) { - case NONE: - break; - case SORT: - localStrategy = "Sort"; - break; - case COMBININGSORT: - localStrategy = "Sort (combining)"; - break; - default: - throw new CompilerException( - "Unknown local strategy " + channel.getLocalStrategy().name()); - } - - if (channel != null - && channel.getLocalStrategyKeys() != null - && channel.getLocalStrategyKeys().size() > 0) { - localStrategy += - " on " - + (channel.getLocalStrategySortOrder() == null - ? channel.getLocalStrategyKeys().toString() - : Utils.createOrdering( - channel.getLocalStrategyKeys(), - channel.getLocalStrategySortOrder()) - .toString()); - } - - if (localStrategy != null) { - writer.print(", \"local_strategy\": \"" + localStrategy + "\""); - } - - if (channel != null && channel.getTempMode() != TempMode.NONE) { - String tempMode = channel.getTempMode().toString(); - writer.print(", \"temp_mode\": \"" + tempMode + "\""); - } - - if (channel != null) { - String exchangeMode = channel.getDataExchangeMode().toString(); - writer.print(", \"exchange_mode\": \"" + exchangeMode + "\""); - } - } - - writer.print('}'); - inputNum++; - } - // finish predecessors - writer.print("\n\t\t]"); - } - - // --------------------------------------------------------------------------------------- - // the part below here is relevant only to plan nodes with concrete strategies, etc - // --------------------------------------------------------------------------------------- - - final PlanNode p = node.getPlanNode(); - if (p == null) { - // finish node - writer.print("\n\t}"); - return true; - } - // local strategy - String locString = null; - if (p.getDriverStrategy() != null) { - switch (p.getDriverStrategy()) { - case NONE: - case BINARY_NO_OP: - break; - - case UNARY_NO_OP: - locString = "No-Op"; - break; - - case MAP: - locString = "Map"; - break; - - case FLAT_MAP: - locString = "FlatMap"; - break; - - case MAP_PARTITION: - locString = "Map Partition"; - break; - - case ALL_REDUCE: - locString = "Reduce All"; - break; - - case ALL_GROUP_REDUCE: - case ALL_GROUP_REDUCE_COMBINE: - locString = "Group Reduce All"; - break; - - case SORTED_REDUCE: - locString = "Sorted Reduce"; - break; - - case SORTED_PARTIAL_REDUCE: - locString = "Sorted Combine/Reduce"; - break; - - case SORTED_GROUP_REDUCE: - locString = "Sorted Group Reduce"; - break; - - case SORTED_GROUP_COMBINE: - locString = "Sorted Combine"; - break; - - case HYBRIDHASH_BUILD_FIRST: - locString = "Hybrid Hash (build: " + child1name + ")"; - break; - case HYBRIDHASH_BUILD_SECOND: - locString = "Hybrid Hash (build: " + child2name + ")"; - break; - - case HYBRIDHASH_BUILD_FIRST_CACHED: - locString = "Hybrid Hash (CACHED) (build: " + child1name + ")"; - break; - case HYBRIDHASH_BUILD_SECOND_CACHED: - locString = "Hybrid Hash (CACHED) (build: " + child2name + ")"; - break; - - case NESTEDLOOP_BLOCKED_OUTER_FIRST: - locString = "Nested Loops (Blocked Outer: " + child1name + ")"; - break; - case NESTEDLOOP_BLOCKED_OUTER_SECOND: - locString = "Nested Loops (Blocked Outer: " + child2name + ")"; - break; - case NESTEDLOOP_STREAMED_OUTER_FIRST: - locString = "Nested Loops (Streamed Outer: " + child1name + ")"; - break; - case NESTEDLOOP_STREAMED_OUTER_SECOND: - locString = "Nested Loops (Streamed Outer: " + child2name + ")"; - break; - - case INNER_MERGE: - locString = "Merge"; - break; - - case CO_GROUP: - locString = "Co-Group"; - break; - - default: - locString = p.getDriverStrategy().name(); - break; - } - - if (locString != null) { - writer.print(",\n\t\t\"driver_strategy\": \""); - writer.print(locString); - writer.print("\""); - } - } - - { - // output node global properties - final GlobalProperties gp = p.getGlobalProperties(); - - writer.print(",\n\t\t\"global_properties\": [\n"); - - addProperty(writer, "Partitioning", gp.getPartitioning().name(), true); - if (gp.getPartitioningFields() != null) { - addProperty(writer, "Partitioned on", gp.getPartitioningFields().toString(), false); - } - if (gp.getPartitioningOrdering() != null) { - addProperty( - writer, - "Partitioning Order", - gp.getPartitioningOrdering().toString(), - false); - } else { - addProperty(writer, "Partitioning Order", "(none)", false); - } - if (n.getUniqueFields() == null || n.getUniqueFields().size() == 0) { - addProperty(writer, "Uniqueness", "not unique", false); - } else { - addProperty(writer, "Uniqueness", n.getUniqueFields().toString(), false); - } - - writer.print("\n\t\t]"); - } - - { - // output node local properties - LocalProperties lp = p.getLocalProperties(); - - writer.print(",\n\t\t\"local_properties\": [\n"); - - if (lp.getOrdering() != null) { - addProperty(writer, "Order", lp.getOrdering().toString(), true); - } else { - addProperty(writer, "Order", "(none)", true); - } - if (lp.getGroupedFields() != null && lp.getGroupedFields().size() > 0) { - addProperty(writer, "Grouped on", lp.getGroupedFields().toString(), false); - } else { - addProperty(writer, "Grouping", "not grouped", false); - } - if (n.getUniqueFields() == null || n.getUniqueFields().size() == 0) { - addProperty(writer, "Uniqueness", "not unique", false); - } else { - addProperty(writer, "Uniqueness", n.getUniqueFields().toString(), false); - } - - writer.print("\n\t\t]"); - } - - // output node size estimates - writer.print(",\n\t\t\"estimates\": [\n"); - - addProperty( - writer, - "Est. Output Size", - n.getEstimatedOutputSize() == -1 - ? "(unknown)" - : formatNumber(n.getEstimatedOutputSize(), "B"), - true); - addProperty( - writer, - "Est. Cardinality", - n.getEstimatedNumRecords() == -1 - ? "(unknown)" - : formatNumber(n.getEstimatedNumRecords()), - false); - - writer.print("\t\t]"); - - // output node cost - if (p.getNodeCosts() != null) { - writer.print(",\n\t\t\"costs\": [\n"); - - addProperty( - writer, - "Network", - p.getNodeCosts().getNetworkCost() == -1 - ? "(unknown)" - : formatNumber(p.getNodeCosts().getNetworkCost(), "B"), - true); - addProperty( - writer, - "Disk I/O", - p.getNodeCosts().getDiskCost() == -1 - ? "(unknown)" - : formatNumber(p.getNodeCosts().getDiskCost(), "B"), - false); - addProperty( - writer, - "CPU", - p.getNodeCosts().getCpuCost() == -1 - ? "(unknown)" - : formatNumber(p.getNodeCosts().getCpuCost(), ""), - false); - - addProperty( - writer, - "Cumulative Network", - p.getCumulativeCosts().getNetworkCost() == -1 - ? "(unknown)" - : formatNumber(p.getCumulativeCosts().getNetworkCost(), "B"), - false); - addProperty( - writer, - "Cumulative Disk I/O", - p.getCumulativeCosts().getDiskCost() == -1 - ? "(unknown)" - : formatNumber(p.getCumulativeCosts().getDiskCost(), "B"), - false); - addProperty( - writer, - "Cumulative CPU", - p.getCumulativeCosts().getCpuCost() == -1 - ? "(unknown)" - : formatNumber(p.getCumulativeCosts().getCpuCost(), ""), - false); - - writer.print("\n\t\t]"); - } - - // output the node compiler hints - if (n.getOperator().getCompilerHints() != null) { - CompilerHints hints = n.getOperator().getCompilerHints(); - CompilerHints defaults = new CompilerHints(); - - String size = - hints.getOutputSize() == defaults.getOutputSize() - ? "(none)" - : String.valueOf(hints.getOutputSize()); - String card = - hints.getOutputCardinality() == defaults.getOutputCardinality() - ? "(none)" - : String.valueOf(hints.getOutputCardinality()); - String width = - hints.getAvgOutputRecordSize() == defaults.getAvgOutputRecordSize() - ? "(none)" - : String.valueOf(hints.getAvgOutputRecordSize()); - String filter = - hints.getFilterFactor() == defaults.getFilterFactor() - ? "(none)" - : String.valueOf(hints.getFilterFactor()); - - writer.print(",\n\t\t\"compiler_hints\": [\n"); - - addProperty(writer, "Output Size (bytes)", size, true); - addProperty(writer, "Output Cardinality", card, false); - addProperty(writer, "Avg. Output Record Size (bytes)", width, false); - addProperty(writer, "Filter Factor", filter, false); - - writer.print("\t\t]"); - } - - // finish node - writer.print("\n\t}"); - return true; - } - - private void addProperty(PrintWriter writer, String name, String value, boolean first) { - if (!first) { - writer.print(",\n"); - } - writer.print("\t\t\t{ \"name\": \""); - writer.print(name); - writer.print("\", \"value\": \""); - writer.print(value); - writer.print("\" }"); - } - - public static String formatNumber(double number) { - return formatNumber(number, ""); - } - - public static String formatNumber(double number, String suffix) { - if (number <= 0.0) { - return String.valueOf(number); - } - - int power = (int) Math.ceil(Math.log10(number)); - - int group = (power - 1) / 3; - if (group >= SIZE_SUFFIXES.length) { - group = SIZE_SUFFIXES.length - 1; - } else if (group < 0) { - group = 0; - } - - // truncate fractional part - int beforeDecimal = power - group * 3; - if (power > beforeDecimal) { - for (int i = power - beforeDecimal; i > 0; i--) { - number /= 10; - } - } - - return group > 0 - ? String.format(Locale.US, "%.2f %s", number, SIZE_SUFFIXES[group]) - : String.format(Locale.US, "%.2f", number); - } - - private static final char[] SIZE_SUFFIXES = {0, 'K', 'M', 'G', 'T'}; -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java deleted file mode 100644 index 9b3fb1c2bf593..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ /dev/null @@ -1,2032 +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.optimizer.plantranslate; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.aggregators.AggregatorRegistry; -import org.apache.flink.api.common.aggregators.AggregatorWithName; -import org.apache.flink.api.common.aggregators.ConvergenceCriterion; -import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.common.operators.util.UserCodeWrapper; -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.api.java.io.BlockingShuffleOutputFormat; -import org.apache.flink.configuration.AlgorithmOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.IterationPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SolutionSetPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plan.WorksetPlanNode; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion; -import org.apache.flink.runtime.iterative.task.IterationHeadTask; -import org.apache.flink.runtime.iterative.task.IterationIntermediateTask; -import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask; -import org.apache.flink.runtime.iterative.task.IterationTailTask; -import org.apache.flink.runtime.jobgraph.DistributionPattern; -import org.apache.flink.runtime.jobgraph.InputOutputFormatContainer; -import org.apache.flink.runtime.jobgraph.InputOutputFormatVertex; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.JobEdge; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphBuilder; -import org.apache.flink.runtime.jobgraph.JobGraphUtils; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.BatchTask; -import org.apache.flink.runtime.operators.CoGroupDriver; -import org.apache.flink.runtime.operators.CoGroupWithSolutionSetFirstDriver; -import org.apache.flink.runtime.operators.CoGroupWithSolutionSetSecondDriver; -import org.apache.flink.runtime.operators.DataSinkTask; -import org.apache.flink.runtime.operators.DataSourceTask; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.JoinDriver; -import org.apache.flink.runtime.operators.JoinWithSolutionSetFirstDriver; -import org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver; -import org.apache.flink.runtime.operators.NoOpDriver; -import org.apache.flink.runtime.operators.chaining.ChainedDriver; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.util.StringUtils; -import org.apache.flink.util.Visitor; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.configuration.AlgorithmOptions.USE_LARGE_RECORDS_HANDLER; -import static org.apache.flink.configuration.ConfigurationUtils.getBooleanConfigOption; -import static org.apache.flink.util.Preconditions.checkState; - -/** - * This component translates the optimizer's resulting {@link - * org.apache.flink.optimizer.plan.OptimizedPlan} to a {@link - * org.apache.flink.runtime.jobgraph.JobGraph}. The translation is not strictly a one-to-one, - * because some nodes from the OptimizedPlan are collapsed into one job vertex. - * - *

This translation does not make any decisions or assumptions. All degrees-of-freedom in the - * execution of the job are made by the Optimizer, so that this translation becomes a deterministic - * mapping. - * - *

The basic method of operation is a top down traversal over the plan graph. On the way down, - * job vertices are created for the plan nodes, on the way back up, the nodes connect their - * predecessors. - */ -public class JobGraphGenerator implements Visitor { - - private static final Logger LOG = LoggerFactory.getLogger(JobGraphGenerator.class); - - public static final String MERGE_ITERATION_AUX_TASKS_KEY = "compiler.merge-iteration-aux"; - - private static final boolean mergeIterationAuxTasks = - GlobalConfiguration.loadConfiguration() - .get(getBooleanConfigOption(MERGE_ITERATION_AUX_TASKS_KEY), false); - - private static final TaskInChain ALREADY_VISITED_PLACEHOLDER = - new TaskInChain(null, null, null, null); - - // ------------------------------------------------------------------------ - - private Map vertices; // a map from optimizer nodes to job vertices - - private Map chainedTasks; // a map from optimizer nodes to job vertices - - private Map iterations; - - private List chainedTasksInSequence; - - private List - auxVertices; // auxiliary vertices which are added during job graph generation - - private final int defaultMaxFan; - - private final float defaultSortSpillingThreshold; - - private final boolean useLargeRecordHandler; - - private int iterationIdEnumerator = 1; - - private IterationPlanNode currentIteration; // the current the enclosing iteration - - private List iterationStack; // stack of enclosing iterations - - private SlotSharingGroup sharingGroup; - - // ------------------------------------------------------------------------ - - /** - * Creates a new job graph generator that uses the default values for its resource - * configuration. - */ - public JobGraphGenerator() { - this.defaultMaxFan = AlgorithmOptions.SPILLING_MAX_FAN.defaultValue(); - this.defaultSortSpillingThreshold = AlgorithmOptions.SORT_SPILLING_THRESHOLD.defaultValue(); - this.useLargeRecordHandler = USE_LARGE_RECORDS_HANDLER.defaultValue(); - } - - public JobGraphGenerator(Configuration config) { - this.defaultMaxFan = config.get(AlgorithmOptions.SPILLING_MAX_FAN); - this.defaultSortSpillingThreshold = config.get(AlgorithmOptions.SORT_SPILLING_THRESHOLD); - this.useLargeRecordHandler = config.get(USE_LARGE_RECORDS_HANDLER); - } - - /** - * Translates a {@link org.apache.flink.optimizer.plan.OptimizedPlan} into a {@link - * org.apache.flink.runtime.jobgraph.JobGraph}. - * - * @param program Optimized plan that is translated into a JobGraph. - * @return JobGraph generated from the plan. - */ - public JobGraph compileJobGraph(OptimizedPlan program) { - return compileJobGraph(program, null); - } - - public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) { - if (program == null) { - throw new NullPointerException( - "Program is null, did you called " + "ExecutionEnvironment.execute()"); - } - - if (jobId == null) { - jobId = JobID.generate(); - } - - this.vertices = new HashMap(); - this.chainedTasks = new HashMap(); - this.chainedTasksInSequence = new ArrayList(); - this.auxVertices = new ArrayList(); - this.iterations = new HashMap(); - this.iterationStack = new ArrayList(); - - this.sharingGroup = new SlotSharingGroup(); - - ExecutionConfig executionConfig = program.getOriginalPlan().getExecutionConfig(); - Configuration jobConfiguration = program.getOriginalPlan().getJobConfiguration(); - - // this starts the traversal that generates the job graph - program.accept(this); - - // sanity check that we are not somehow in an iteration at the end - if (this.currentIteration != null) { - throw new CompilerException( - "The graph translation ended prematurely, leaving an unclosed iteration."); - } - - // finalize the iterations - for (IterationDescriptor iteration : this.iterations.values()) { - if (iteration.getIterationNode() instanceof BulkIterationPlanNode) { - finalizeBulkIteration(iteration); - } else if (iteration.getIterationNode() instanceof WorksetIterationPlanNode) { - finalizeWorksetIteration(iteration); - } else { - throw new CompilerException(); - } - } - - // now that the traversal is done, we have the chained tasks write their configs into their - // parents' configurations - for (TaskInChain tic : this.chainedTasksInSequence) { - TaskConfig t = new TaskConfig(tic.getContainingVertex().getConfiguration()); - t.addChainedTask(tic.getChainedTask(), tic.getTaskConfig(), tic.getTaskName()); - } - - // ----- attach the additional info to the job vertices, for display in the runtime monitor - - attachOperatorNamesAndDescriptions(); - - // ----------- finalize the job graph ----------- - - for (JobVertex vertex : this.auxVertices) { - vertex.setSlotSharingGroup(sharingGroup); - } - - final Map userArtifacts = - JobGraphUtils.prepareUserArtifactEntries( - program.getOriginalPlan().getCachedFiles().stream() - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)), - jobId); - - // create the job graph object - final JobGraph graph; - try { - graph = - JobGraphBuilder.newBatchJobGraphBuilder() - .setJobId(jobId) - .setJobName(program.getJobName()) - .setExecutionConfig(executionConfig) - .setJobConfiguration(jobConfiguration) - .addJobVertices(vertices.values()) - .addJobVertices(auxVertices) - .addUserArtifacts(userArtifacts) - .build(); - - if (executionConfig.getSchedulerType().isPresent() - && executionConfig.getSchedulerType().get() - == JobManagerOptions.SchedulerType.AdaptiveBatch) { - graph.setDynamic(true); - } - - } catch (IOException e) { - throw new CompilerException( - "Could not serialize the ExecutionConfig." - + "This indicates that non-serializable types (like custom serializers) were registered"); - } - - // release all references again - this.vertices = null; - this.chainedTasks = null; - this.chainedTasksInSequence = null; - this.auxVertices = null; - this.iterations = null; - this.iterationStack = null; - - // return job graph - return graph; - } - - /** - * This methods implements the pre-visiting during a depth-first traversal. It create the job - * vertex and sets local strategy. - * - * @param node The node that is currently processed. - * @return True, if the visitor should descend to the node's children, false if not. - * @see org.apache.flink.util.Visitor#preVisit(org.apache.flink.util.Visitable) - */ - @Override - public boolean preVisit(PlanNode node) { - // check if we have visited this node before. in non-tree graphs, this happens - if (this.vertices.containsKey(node) - || this.chainedTasks.containsKey(node) - || this.iterations.containsKey(node)) { - // return false to prevent further descend - return false; - } - - // the vertex to be created for the current node - final JobVertex vertex; - try { - if (node instanceof SinkPlanNode) { - vertex = createDataSinkVertex((SinkPlanNode) node); - } else if (node instanceof SourcePlanNode) { - vertex = createDataSourceVertex((SourcePlanNode) node); - } else if (node instanceof BulkIterationPlanNode) { - BulkIterationPlanNode iterationNode = (BulkIterationPlanNode) node; - // for the bulk iteration, we skip creating anything for now. we create the graph - // for the step function in the post visit. - - // check that the root of the step function has the same parallelism as the - // iteration. - // because the tail must have the same parallelism as the head, we can only merge - // the last - // operator with the tail, if they have the same parallelism. not merging is - // currently not - // implemented - PlanNode root = iterationNode.getRootOfStepFunction(); - if (root.getParallelism() != node.getParallelism()) { - throw new CompilerException( - "Error: The final operator of the step " - + "function has a different parallelism than the iteration operator itself."); - } - - IterationDescriptor descr = - new IterationDescriptor(iterationNode, this.iterationIdEnumerator++); - this.iterations.put(iterationNode, descr); - vertex = null; - } else if (node instanceof WorksetIterationPlanNode) { - WorksetIterationPlanNode iterationNode = (WorksetIterationPlanNode) node; - - // we have the same constraints as for the bulk iteration - PlanNode nextWorkSet = iterationNode.getNextWorkSetPlanNode(); - PlanNode solutionSetDelta = iterationNode.getSolutionSetDeltaPlanNode(); - - if (nextWorkSet.getParallelism() != node.getParallelism()) { - throw new CompilerException( - "It is currently not supported that the final operator of the step " - + "function has a different parallelism than the iteration operator itself."); - } - if (solutionSetDelta.getParallelism() != node.getParallelism()) { - throw new CompilerException( - "It is currently not supported that the final operator of the step " - + "function has a different parallelism than the iteration operator itself."); - } - - IterationDescriptor descr = - new IterationDescriptor(iterationNode, this.iterationIdEnumerator++); - this.iterations.put(iterationNode, descr); - vertex = null; - } else if (node instanceof SingleInputPlanNode) { - vertex = createSingleInputVertex((SingleInputPlanNode) node); - } else if (node instanceof DualInputPlanNode) { - vertex = createDualInputVertex((DualInputPlanNode) node); - } else if (node instanceof NAryUnionPlanNode) { - // skip the union for now - vertex = null; - } else if (node instanceof BulkPartialSolutionPlanNode) { - // create a head node (or not, if it is merged into its successor) - vertex = createBulkIterationHead((BulkPartialSolutionPlanNode) node); - } else if (node instanceof SolutionSetPlanNode) { - // this represents an access into the solution set index. - // we do not create a vertex for the solution set here (we create the head at the - // workset place holder) - - // we adjust the joins / cogroups that go into the solution set here - for (Channel c : node.getOutgoingChannels()) { - DualInputPlanNode target = (DualInputPlanNode) c.getTarget(); - JobVertex accessingVertex = this.vertices.get(target); - TaskConfig conf = new TaskConfig(accessingVertex.getConfiguration()); - int inputNum = c == target.getInput1() ? 0 : c == target.getInput2() ? 1 : -1; - - // sanity checks - if (inputNum == -1) { - throw new CompilerException(); - } - - // adjust the driver - if (conf.getDriver().equals(JoinDriver.class)) { - conf.setDriver( - inputNum == 0 - ? JoinWithSolutionSetFirstDriver.class - : JoinWithSolutionSetSecondDriver.class); - } else if (conf.getDriver().equals(CoGroupDriver.class)) { - conf.setDriver( - inputNum == 0 - ? CoGroupWithSolutionSetFirstDriver.class - : CoGroupWithSolutionSetSecondDriver.class); - } else { - throw new CompilerException( - "Found join with solution set using incompatible operator (only Join/CoGroup are valid)."); - } - } - - // make sure we do not visit this node again. for that, we add a 'already seen' - // entry into one of the sets - this.chainedTasks.put(node, ALREADY_VISITED_PLACEHOLDER); - - vertex = null; - } else if (node instanceof WorksetPlanNode) { - // create the iteration head here - vertex = createWorksetIterationHead((WorksetPlanNode) node); - } else { - throw new CompilerException("Unrecognized node type: " + node.getClass().getName()); - } - } catch (Exception e) { - throw new CompilerException( - "Error translating node '" + node + "': " + e.getMessage(), e); - } - - // check if a vertex was created, or if it was chained or skipped - if (vertex != null) { - // set parallelism - int pd = node.getParallelism(); - vertex.setParallelism(pd); - vertex.setMaxParallelism(pd); - - vertex.setSlotSharingGroup(sharingGroup); - - // check whether this vertex is part of an iteration step function - if (this.currentIteration != null) { - // check that the task has the same parallelism as the iteration as such - PlanNode iterationNode = (PlanNode) this.currentIteration; - if (iterationNode.getParallelism() < pd) { - throw new CompilerException( - "Error: All functions that are part of an iteration must have the same, or a lower, parallelism than the iteration operator."); - } - - // store the id of the iterations the step functions participate in - IterationDescriptor descr = this.iterations.get(this.currentIteration); - new TaskConfig(vertex.getConfiguration()).setIterationId(descr.getId()); - } - - // store in the map - this.vertices.put(node, vertex); - } - - // returning true causes deeper descend - return true; - } - - /** - * This method implements the post-visit during the depth-first traversal. When the post visit - * happens, all of the descendants have been processed, so this method connects all of the - * current node's predecessors to the current node. - * - * @param node The node currently processed during the post-visit. - * @see org.apache.flink.util.Visitor#postVisit(org.apache.flink.util.Visitable) t - */ - @Override - public void postVisit(PlanNode node) { - try { - // --------- check special cases for which we handle post visit differently ---------- - - // skip data source node (they have no inputs) - // also, do nothing for union nodes, we connect them later when gathering the inputs for - // a task - // solution sets have no input. the initial solution set input is connected when the - // iteration node is in its postVisit - if (node instanceof SourcePlanNode - || node instanceof NAryUnionPlanNode - || node instanceof SolutionSetPlanNode) { - return; - } - - // if this is a blocking shuffle vertex, we add one IntermediateDataSetID to its - // predecessor and return - if (checkAndConfigurePersistentIntermediateResult(node)) { - return; - } - - // check if we have an iteration. in that case, translate the step function now - if (node instanceof IterationPlanNode) { - // prevent nested iterations - if (node.isOnDynamicPath()) { - throw new CompilerException( - "Nested Iterations are not possible at the moment!"); - } - - // if we recursively go into an iteration (because the constant path of one - // iteration contains - // another one), we push the current one onto the stack - if (this.currentIteration != null) { - this.iterationStack.add(this.currentIteration); - } - - this.currentIteration = (IterationPlanNode) node; - this.currentIteration.acceptForStepFunction(this); - - // pop the current iteration from the stack - if (this.iterationStack.isEmpty()) { - this.currentIteration = null; - } else { - this.currentIteration = - this.iterationStack.remove(this.iterationStack.size() - 1); - } - - // inputs for initial bulk partial solution or initial workset are already connected - // to the iteration head in the head's post visit. - // connect the initial solution set now. - if (node instanceof WorksetIterationPlanNode) { - // connect the initial solution set - WorksetIterationPlanNode wsNode = (WorksetIterationPlanNode) node; - JobVertex headVertex = this.iterations.get(wsNode).getHeadTask(); - TaskConfig headConfig = new TaskConfig(headVertex.getConfiguration()); - int inputIndex = headConfig.getDriverStrategy().getNumInputs(); - headConfig.setIterationHeadSolutionSetInputIndex(inputIndex); - translateChannel( - wsNode.getInitialSolutionSetInput(), - inputIndex, - headVertex, - headConfig, - false); - } - - return; - } - - final JobVertex targetVertex = this.vertices.get(node); - - // --------- Main Path: Translation of channels ---------- - // - // There are two paths of translation: One for chained tasks (or merged tasks in - // general), - // which do not have their own task vertex. The other for tasks that have their own - // vertex, - // or are the primary task in a vertex (to which the others are chained). - - // check whether this node has its own task, or is merged with another one - if (targetVertex == null) { - // node's task is merged with another task. it is either chained, of a merged head - // vertex - // from an iteration - final TaskInChain chainedTask; - if ((chainedTask = this.chainedTasks.get(node)) != null) { - // Chained Task. Sanity check first... - final Iterator inConns = node.getInputs().iterator(); - if (!inConns.hasNext()) { - throw new CompilerException("Bug: Found chained task with no input."); - } - final Channel inConn = inConns.next(); - - if (inConns.hasNext()) { - throw new CompilerException( - "Bug: Found a chained task with more than one input!"); - } - if (inConn.getLocalStrategy() != null - && inConn.getLocalStrategy() != LocalStrategy.NONE) { - throw new CompilerException( - "Bug: Found a chained task with an input local strategy."); - } - if (inConn.getShipStrategy() != null - && inConn.getShipStrategy() != ShipStrategyType.FORWARD) { - throw new CompilerException( - "Bug: Found a chained task with an input ship strategy other than FORWARD."); - } - - JobVertex container = chainedTask.getContainingVertex(); - - if (container == null) { - final PlanNode sourceNode = inConn.getSource(); - container = this.vertices.get(sourceNode); - if (container == null) { - // predecessor is itself chained - container = this.chainedTasks.get(sourceNode).getContainingVertex(); - if (container == null) { - throw new IllegalStateException( - "Bug: Chained task predecessor has not been assigned its containing vertex."); - } - } else { - // predecessor is a proper task job vertex and this is the first chained - // task. add a forward connection entry. - new TaskConfig(container.getConfiguration()) - .addOutputShipStrategy(ShipStrategyType.FORWARD); - } - chainedTask.setContainingVertex(container); - } - - // add info about the input serializer type - chainedTask.getTaskConfig().setInputSerializer(inConn.getSerializer(), 0); - - // update name of container task - String containerTaskName = container.getName(); - if (containerTaskName.startsWith("CHAIN ")) { - container.setName(containerTaskName + " -> " + chainedTask.getTaskName()); - } else { - container.setName( - "CHAIN " + containerTaskName + " -> " + chainedTask.getTaskName()); - } - - // update resource of container task - container.setResources( - container.getMinResources().merge(node.getMinResources()), - container.getPreferredResources().merge(node.getPreferredResources())); - - this.chainedTasksInSequence.add(chainedTask); - return; - } else if (node instanceof BulkPartialSolutionPlanNode - || node instanceof WorksetPlanNode) { - // merged iteration head task. the task that the head is merged with will take - // care of it - return; - } else { - throw new CompilerException("Bug: Unrecognized merged task vertex."); - } - } - - // -------- Here, we translate non-chained tasks ------------- - - if (this.currentIteration != null) { - JobVertex head = this.iterations.get(this.currentIteration).getHeadTask(); - // Exclude static code paths from the co-location constraint, because otherwise - // their execution determines the deployment slots of the co-location group - if (node.isOnDynamicPath()) { - targetVertex.setStrictlyCoLocatedWith(head); - } - } - - // create the config that will contain all the description of the inputs - final TaskConfig targetVertexConfig = new TaskConfig(targetVertex.getConfiguration()); - - // get the inputs. if this node is the head of an iteration, we obtain the inputs from - // the - // enclosing iteration node, because the inputs are the initial inputs to the iteration. - final Iterator inConns; - if (node instanceof BulkPartialSolutionPlanNode) { - inConns = - ((BulkPartialSolutionPlanNode) node) - .getContainingIterationNode() - .getInputs() - .iterator(); - // because the partial solution has its own vertex, is has only one (logical) input. - // note this in the task configuration - targetVertexConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0); - } else if (node instanceof WorksetPlanNode) { - WorksetPlanNode wspn = (WorksetPlanNode) node; - // input that is the initial workset - inConns = - Collections.singleton(wspn.getContainingIterationNode().getInput2()) - .iterator(); - - // because we have a stand-alone (non-merged) workset iteration head, the initial - // workset will - // be input 0 and the solution set will be input 1 - targetVertexConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0); - targetVertexConfig.setIterationHeadSolutionSetInputIndex(1); - } else { - inConns = node.getInputs().iterator(); - } - if (!inConns.hasNext()) { - throw new CompilerException("Bug: Found a non-source task with no input."); - } - - int inputIndex = 0; - while (inConns.hasNext()) { - Channel input = inConns.next(); - inputIndex += - translateChannel( - input, inputIndex, targetVertex, targetVertexConfig, false); - } - // broadcast variables - int broadcastInputIndex = 0; - for (NamedChannel broadcastInput : node.getBroadcastInputs()) { - int broadcastInputIndexDelta = - translateChannel( - broadcastInput, - broadcastInputIndex, - targetVertex, - targetVertexConfig, - true); - targetVertexConfig.setBroadcastInputName( - broadcastInput.getName(), broadcastInputIndex); - targetVertexConfig.setBroadcastInputSerializer( - broadcastInput.getSerializer(), broadcastInputIndex); - broadcastInputIndex += broadcastInputIndexDelta; - } - } catch (Exception e) { - throw new CompilerException( - "An error occurred while translating the optimized plan to a JobGraph: " - + e.getMessage(), - e); - } - } - - private int translateChannel( - Channel input, - int inputIndex, - JobVertex targetVertex, - TaskConfig targetVertexConfig, - boolean isBroadcast) - throws Exception { - final PlanNode inputPlanNode = input.getSource(); - final Iterator allInChannels; - - if (inputPlanNode instanceof NAryUnionPlanNode) { - allInChannels = ((NAryUnionPlanNode) inputPlanNode).getListOfInputs().iterator(); - - // If the union node has a batch data exchange, we have to adopt the exchange mode of - // the inputs of the union as well, because the optimizer has a separate union - // node, which does not exist in the JobGraph. Otherwise, this can result in - // deadlocks when closing a branching flow at runtime. - for (Channel in : inputPlanNode.getInputs()) { - if (input.getDataExchangeMode().equals(DataExchangeMode.BATCH)) { - in.setDataExchangeMode(DataExchangeMode.BATCH); - } - if (isBroadcast) { - in.setShipStrategy(ShipStrategyType.BROADCAST, in.getDataExchangeMode()); - } - } - - // The outgoing connection of an NAryUnion must be a forward connection. - if (input.getShipStrategy() != ShipStrategyType.FORWARD && !isBroadcast) { - throw new CompilerException( - "Optimized plan contains Union with non-forward outgoing ship strategy."); - } - - } else if (inputPlanNode instanceof BulkPartialSolutionPlanNode) { - if (this.vertices.get(inputPlanNode) == null) { - // merged iteration head - final BulkPartialSolutionPlanNode pspn = - (BulkPartialSolutionPlanNode) inputPlanNode; - final BulkIterationPlanNode iterationNode = pspn.getContainingIterationNode(); - - // check if the iteration's input is a union - if (iterationNode.getInput().getSource() instanceof NAryUnionPlanNode) { - allInChannels = (iterationNode.getInput().getSource()).getInputs().iterator(); - } else { - allInChannels = Collections.singletonList(iterationNode.getInput()).iterator(); - } - - // also, set the index of the gate with the partial solution - targetVertexConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(inputIndex); - } else { - // standalone iteration head - allInChannels = Collections.singletonList(input).iterator(); - } - } else if (inputPlanNode instanceof WorksetPlanNode) { - if (this.vertices.get(inputPlanNode) == null) { - // merged iteration head - final WorksetPlanNode wspn = (WorksetPlanNode) inputPlanNode; - final WorksetIterationPlanNode iterationNode = wspn.getContainingIterationNode(); - - // check if the iteration's input is a union - if (iterationNode.getInput2().getSource() instanceof NAryUnionPlanNode) { - allInChannels = (iterationNode.getInput2().getSource()).getInputs().iterator(); - } else { - allInChannels = Collections.singletonList(iterationNode.getInput2()).iterator(); - } - - // also, set the index of the gate with the partial solution - targetVertexConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(inputIndex); - } else { - // standalone iteration head - allInChannels = Collections.singletonList(input).iterator(); - } - } else if (inputPlanNode instanceof SolutionSetPlanNode) { - // for now, skip connections with the solution set node, as this is a local index access - // (later to be parameterized here) - // rather than a vertex connection - return 0; - } else { - allInChannels = Collections.singletonList(input).iterator(); - } - - // check that the type serializer is consistent - TypeSerializerFactory typeSerFact = null; - - // accounting for channels on the dynamic path - int numChannelsTotal = 0; - int numChannelsDynamicPath = 0; - int numDynamicSenderTasksTotal = 0; - - // expand the channel to all the union channels, in case there is a union operator at its - // source - while (allInChannels.hasNext()) { - final Channel inConn = allInChannels.next(); - - // sanity check the common serializer - if (typeSerFact == null) { - typeSerFact = inConn.getSerializer(); - } else if (!typeSerFact.equals(inConn.getSerializer())) { - throw new CompilerException("Conflicting types in union operator."); - } - - final PlanNode sourceNode = inConn.getSource(); - JobVertex sourceVertex = this.vertices.get(sourceNode); - TaskConfig sourceVertexConfig; - - if (sourceVertex == null) { - // this predecessor is chained to another task or an iteration - final TaskInChain chainedTask; - final IterationDescriptor iteration; - if ((chainedTask = this.chainedTasks.get(sourceNode)) != null) { - // push chained task - if (chainedTask.getContainingVertex() == null) { - throw new IllegalStateException( - "Bug: Chained task has not been assigned its containing vertex when connecting."); - } - sourceVertex = chainedTask.getContainingVertex(); - sourceVertexConfig = chainedTask.getTaskConfig(); - } else if ((iteration = this.iterations.get(sourceNode)) != null) { - // predecessor is an iteration - sourceVertex = iteration.getHeadTask(); - sourceVertexConfig = iteration.getHeadFinalResultConfig(); - } else { - throw new CompilerException( - "Bug: Could not resolve source node for a channel."); - } - } else { - // predecessor is its own vertex - sourceVertexConfig = new TaskConfig(sourceVertex.getConfiguration()); - } - DistributionPattern pattern = - connectJobVertices( - inConn, - inputIndex, - sourceVertex, - sourceVertexConfig, - targetVertex, - targetVertexConfig, - isBroadcast); - - // accounting on channels and senders - numChannelsTotal++; - if (inConn.isOnDynamicPath()) { - numChannelsDynamicPath++; - numDynamicSenderTasksTotal += - getNumberOfSendersPerReceiver( - pattern, - sourceVertex.getParallelism(), - targetVertex.getParallelism()); - } - } - - // for the iterations, check that the number of dynamic channels is the same as the number - // of channels for this logical input. this condition is violated at the moment, if there - // is a union between nodes on the static and nodes on the dynamic path - if (numChannelsDynamicPath > 0 && numChannelsTotal != numChannelsDynamicPath) { - throw new CompilerException( - "Error: It is currently not supported to union between dynamic and static path in an iteration."); - } - if (numDynamicSenderTasksTotal > 0) { - if (isBroadcast) { - targetVertexConfig.setBroadcastGateIterativeWithNumberOfEventsUntilInterrupt( - inputIndex, numDynamicSenderTasksTotal); - } else { - targetVertexConfig.setGateIterativeWithNumberOfEventsUntilInterrupt( - inputIndex, numDynamicSenderTasksTotal); - } - } - - // the local strategy is added only once. in non-union case that is the actual edge, - // in the union case, it is the edge between union and the target node - addLocalInfoFromChannelToConfig(input, targetVertexConfig, inputIndex, isBroadcast); - return 1; - } - - private int getNumberOfSendersPerReceiver( - DistributionPattern pattern, int numSenders, int numReceivers) { - if (pattern == DistributionPattern.ALL_TO_ALL) { - return numSenders; - } else if (pattern == DistributionPattern.POINTWISE) { - if (numSenders != numReceivers) { - if (numReceivers == 1) { - return numSenders; - } else if (numSenders == 1) { - return 1; - } else { - throw new CompilerException( - "Error: A changing parallelism is currently " - + "not supported between tasks within an iteration."); - } - } else { - return 1; - } - } else { - throw new CompilerException("Unknown distribution pattern for channels: " + pattern); - } - } - - // ------------------------------------------------------------------------ - // Methods for creating individual vertices - // ------------------------------------------------------------------------ - - private JobVertex createSingleInputVertex(SingleInputPlanNode node) throws CompilerException { - final String taskName = node.getNodeName(); - final DriverStrategy ds = node.getDriverStrategy(); - - // check, whether chaining is possible - boolean chaining; - { - Channel inConn = node.getInput(); - PlanNode pred = inConn.getSource(); - chaining = - ds.getPushChainDriverClass() != null - && !(pred instanceof NAryUnionPlanNode) - && // first op after union is stand-alone, because union is merged - !(pred instanceof BulkPartialSolutionPlanNode) - && // partial solution merges anyways - !(pred instanceof WorksetPlanNode) - && // workset merges anyways - !(pred instanceof IterationPlanNode) - && // cannot chain with iteration heads currently - inConn.getShipStrategy() == ShipStrategyType.FORWARD - && inConn.getLocalStrategy() == LocalStrategy.NONE - && pred.getOutgoingChannels().size() == 1 - && node.getParallelism() == pred.getParallelism() - && node.getBroadcastInputs().isEmpty(); - - // cannot chain the nodes that produce the next workset or the next solution set, if - // they are not the - // in a tail - if (this.currentIteration instanceof WorksetIterationPlanNode - && node.getOutgoingChannels().size() > 0) { - WorksetIterationPlanNode wspn = (WorksetIterationPlanNode) this.currentIteration; - if (wspn.getSolutionSetDeltaPlanNode() == pred - || wspn.getNextWorkSetPlanNode() == pred) { - chaining = false; - } - } - // cannot chain the nodes that produce the next workset in a bulk iteration if a - // termination criterion follows - if (this.currentIteration instanceof BulkIterationPlanNode) { - BulkIterationPlanNode wspn = (BulkIterationPlanNode) this.currentIteration; - if (node == wspn.getRootOfTerminationCriterion() - && wspn.getRootOfStepFunction() == pred) { - chaining = false; - } else if (node.getOutgoingChannels().size() > 0 - && (wspn.getRootOfStepFunction() == pred - || wspn.getRootOfTerminationCriterion() == pred)) { - chaining = false; - } - } - } - - final JobVertex vertex; - final TaskConfig config; - - if (chaining) { - vertex = null; - config = new TaskConfig(new Configuration()); - this.chainedTasks.put( - node, new TaskInChain(node, ds.getPushChainDriverClass(), config, taskName)); - } else { - // create task vertex - vertex = new JobVertex(taskName); - vertex.setResources(node.getMinResources(), node.getPreferredResources()); - vertex.setInvokableClass( - (this.currentIteration != null && node.isOnDynamicPath()) - ? IterationIntermediateTask.class - : BatchTask.class); - - config = new TaskConfig(vertex.getConfiguration()); - config.setDriver(ds.getDriverClass()); - } - - // set user code - config.setStubWrapper(node.getProgramOperator().getUserCodeWrapper()); - config.setStubParameters(node.getProgramOperator().getParameters()); - - // set the driver strategy - config.setDriverStrategy(ds); - for (int i = 0; i < ds.getNumRequiredComparators(); i++) { - config.setDriverComparator(node.getComparator(i), i); - } - // assign memory, file-handles, etc. - assignDriverResources(node, config); - return vertex; - } - - private JobVertex createDualInputVertex(DualInputPlanNode node) throws CompilerException { - final String taskName = node.getNodeName(); - final DriverStrategy ds = node.getDriverStrategy(); - final JobVertex vertex = new JobVertex(taskName); - final TaskConfig config = new TaskConfig(vertex.getConfiguration()); - vertex.setResources(node.getMinResources(), node.getPreferredResources()); - vertex.setInvokableClass( - (this.currentIteration != null && node.isOnDynamicPath()) - ? IterationIntermediateTask.class - : BatchTask.class); - - // set user code - config.setStubWrapper(node.getProgramOperator().getUserCodeWrapper()); - config.setStubParameters(node.getProgramOperator().getParameters()); - - // set the driver strategy - config.setDriver(ds.getDriverClass()); - config.setDriverStrategy(ds); - if (node.getComparator1() != null) { - config.setDriverComparator(node.getComparator1(), 0); - } - if (node.getComparator2() != null) { - config.setDriverComparator(node.getComparator2(), 1); - } - if (node.getPairComparator() != null) { - config.setDriverPairComparator(node.getPairComparator()); - } - - // assign memory, file-handles, etc. - assignDriverResources(node, config); - return vertex; - } - - private JobVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException { - final InputOutputFormatVertex vertex = new InputOutputFormatVertex(node.getNodeName()); - final TaskConfig config = new TaskConfig(vertex.getConfiguration()); - - final OperatorID operatorID = new OperatorID(); - - vertex.setResources(node.getMinResources(), node.getPreferredResources()); - vertex.setInvokableClass(DataSourceTask.class); - vertex.setFormatDescription( - operatorID, - getDescriptionForUserCode(node.getProgramOperator().getUserCodeWrapper())); - - // set user code - new InputOutputFormatContainer(Thread.currentThread().getContextClassLoader()) - .addInputFormat( - operatorID, - (UserCodeWrapper>) - node.getProgramOperator().getUserCodeWrapper()) - .addParameters(operatorID, node.getProgramOperator().getParameters()) - .write(config); - - config.setOutputSerializer(node.getSerializer()); - return vertex; - } - - private JobVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException { - final InputOutputFormatVertex vertex = new InputOutputFormatVertex(node.getNodeName()); - final TaskConfig config = new TaskConfig(vertex.getConfiguration()); - - final OperatorID operatorID = new OperatorID(); - - vertex.setResources(node.getMinResources(), node.getPreferredResources()); - vertex.setInvokableClass(DataSinkTask.class); - vertex.setFormatDescription( - operatorID, - getDescriptionForUserCode(node.getProgramOperator().getUserCodeWrapper())); - - // set user code - new InputOutputFormatContainer(Thread.currentThread().getContextClassLoader()) - .addOutputFormat( - operatorID, - (UserCodeWrapper>) - node.getProgramOperator().getUserCodeWrapper()) - .addParameters(operatorID, node.getProgramOperator().getParameters()) - .write(config); - - return vertex; - } - - private JobVertex createBulkIterationHead(BulkPartialSolutionPlanNode pspn) { - // get the bulk iteration that corresponds to this partial solution node - final BulkIterationPlanNode iteration = pspn.getContainingIterationNode(); - - // check whether we need an individual vertex for the partial solution, or whether we - // attach ourselves to the vertex of the parent node. We can combine the head with a node of - // the step function, if - // 1) There is one parent that the partial solution connects to via a forward pattern and no - // local strategy - // 2) parallelism and the number of subtasks per instance does not change - // 3) That successor is not a union - // 4) That successor is not itself the last node of the step function - // 5) There is no local strategy on the edge for the initial partial solution, as - // this translates to a local strategy that would only be executed in the first iteration - - final boolean merge; - if (mergeIterationAuxTasks && pspn.getOutgoingChannels().size() == 1) { - final Channel c = pspn.getOutgoingChannels().get(0); - final PlanNode successor = c.getTarget(); - merge = - c.getShipStrategy() == ShipStrategyType.FORWARD - && c.getLocalStrategy() == LocalStrategy.NONE - && c.getTempMode() == TempMode.NONE - && successor.getParallelism() == pspn.getParallelism() - && !(successor instanceof NAryUnionPlanNode) - && successor != iteration.getRootOfStepFunction() - && iteration.getInput().getLocalStrategy() == LocalStrategy.NONE; - } else { - merge = false; - } - - // create or adopt the head vertex - final JobVertex toReturn; - final JobVertex headVertex; - final TaskConfig headConfig; - if (merge) { - final PlanNode successor = pspn.getOutgoingChannels().get(0).getTarget(); - headVertex = this.vertices.get(successor); - - if (headVertex == null) { - throw new CompilerException( - "Bug: Trying to merge solution set with its successor, but successor has not been created."); - } - - // reset the vertex type to iteration head - headVertex.setInvokableClass(IterationHeadTask.class); - headConfig = new TaskConfig(headVertex.getConfiguration()); - toReturn = null; - } else { - // instantiate the head vertex and give it a no-op driver as the driver strategy. - // everything else happens in the post visit, after the input (the initial partial - // solution) - // is connected. - headVertex = new JobVertex("PartialSolution (" + iteration.getNodeName() + ")"); - headVertex.setResources(iteration.getMinResources(), iteration.getPreferredResources()); - headVertex.setInvokableClass(IterationHeadTask.class); - headConfig = new TaskConfig(headVertex.getConfiguration()); - headConfig.setDriver(NoOpDriver.class); - toReturn = headVertex; - } - - // create the iteration descriptor and the iteration to it - IterationDescriptor descr = this.iterations.get(iteration); - if (descr == null) { - throw new CompilerException( - "Bug: Iteration descriptor was not created at when translating the iteration node."); - } - descr.setHeadTask(headVertex, headConfig); - - return toReturn; - } - - private JobVertex createWorksetIterationHead(WorksetPlanNode wspn) { - // get the bulk iteration that corresponds to this partial solution node - final WorksetIterationPlanNode iteration = wspn.getContainingIterationNode(); - - // check whether we need an individual vertex for the partial solution, or whether we - // attach ourselves to the vertex of the parent node. We can combine the head with a node of - // the step function, if - // 1) There is one parent that the partial solution connects to via a forward pattern and no - // local strategy - // 2) parallelism and the number of subtasks per instance does not change - // 3) That successor is not a union - // 4) That successor is not itself the last node of the step function - // 5) There is no local strategy on the edge for the initial workset, as - // this translates to a local strategy that would only be executed in the first superstep - - final boolean merge; - if (mergeIterationAuxTasks && wspn.getOutgoingChannels().size() == 1) { - final Channel c = wspn.getOutgoingChannels().get(0); - final PlanNode successor = c.getTarget(); - merge = - c.getShipStrategy() == ShipStrategyType.FORWARD - && c.getLocalStrategy() == LocalStrategy.NONE - && c.getTempMode() == TempMode.NONE - && successor.getParallelism() == wspn.getParallelism() - && !(successor instanceof NAryUnionPlanNode) - && successor != iteration.getNextWorkSetPlanNode() - && iteration.getInitialWorksetInput().getLocalStrategy() - == LocalStrategy.NONE; - } else { - merge = false; - } - - // create or adopt the head vertex - final JobVertex toReturn; - final JobVertex headVertex; - final TaskConfig headConfig; - if (merge) { - final PlanNode successor = wspn.getOutgoingChannels().get(0).getTarget(); - headVertex = this.vertices.get(successor); - - if (headVertex == null) { - throw new CompilerException( - "Bug: Trying to merge solution set with its sucessor, but successor has not been created."); - } - - // reset the vertex type to iteration head - headVertex.setInvokableClass(IterationHeadTask.class); - headConfig = new TaskConfig(headVertex.getConfiguration()); - toReturn = null; - } else { - // instantiate the head vertex and give it a no-op driver as the driver strategy. - // everything else happens in the post visit, after the input (the initial partial - // solution) - // is connected. - headVertex = new JobVertex("IterationHead(" + iteration.getNodeName() + ")"); - headVertex.setResources(iteration.getMinResources(), iteration.getPreferredResources()); - headVertex.setInvokableClass(IterationHeadTask.class); - headConfig = new TaskConfig(headVertex.getConfiguration()); - headConfig.setDriver(NoOpDriver.class); - toReturn = headVertex; - } - - headConfig.setSolutionSetUnmanaged( - iteration.getIterationNode().getIterationContract().isSolutionSetUnManaged()); - - // create the iteration descriptor and the iteration to it - IterationDescriptor descr = this.iterations.get(iteration); - if (descr == null) { - throw new CompilerException( - "Bug: Iteration descriptor was not created at when translating the iteration node."); - } - descr.setHeadTask(headVertex, headConfig); - - return toReturn; - } - - private void assignDriverResources(PlanNode node, TaskConfig config) { - final double relativeMem = node.getRelativeMemoryPerSubTask(); - if (relativeMem > 0) { - config.setRelativeMemoryDriver(relativeMem); - config.setFilehandlesDriver(this.defaultMaxFan); - config.setSpillingThresholdDriver(this.defaultSortSpillingThreshold); - } - } - - private void assignLocalStrategyResources(Channel c, TaskConfig config, int inputNum) { - if (c.getRelativeMemoryLocalStrategy() > 0) { - config.setRelativeMemoryInput(inputNum, c.getRelativeMemoryLocalStrategy()); - config.setFilehandlesInput(inputNum, this.defaultMaxFan); - config.setSpillingThresholdInput(inputNum, this.defaultSortSpillingThreshold); - config.setUseLargeRecordHandler(this.useLargeRecordHandler); - } - } - - private boolean checkAndConfigurePersistentIntermediateResult(PlanNode node) { - if (!(node instanceof SinkPlanNode)) { - return false; - } - - final Object userCodeObject = - node.getProgramOperator().getUserCodeWrapper().getUserCodeObject(); - if (!(userCodeObject instanceof BlockingShuffleOutputFormat)) { - return false; - } - - final Iterator inputIterator = node.getInputs().iterator(); - checkState(inputIterator.hasNext(), "SinkPlanNode must have a input."); - - final PlanNode predecessorNode = inputIterator.next().getSource(); - final JobVertex predecessorVertex = - (vertices.containsKey(predecessorNode)) - ? vertices.get(predecessorNode) - : chainedTasks.get(predecessorNode).getContainingVertex(); - - checkState( - predecessorVertex != null, - "Bug: Chained task has not been assigned its containing vertex when connecting."); - - predecessorVertex.getOrCreateResultDataSet( - // use specified intermediateDataSetID - new IntermediateDataSetID( - ((BlockingShuffleOutputFormat) userCodeObject).getIntermediateDataSetId()), - ResultPartitionType.BLOCKING_PERSISTENT); - - // remove this node so the OutputFormatVertex will not shown in the final JobGraph. - vertices.remove(node); - return true; - } - - // ------------------------------------------------------------------------ - // Connecting Vertices - // ------------------------------------------------------------------------ - - /** - * NOTE: The channel for global and local strategies are different if we connect a union. The - * global strategy channel is then the channel into the union node, the local strategy channel - * the one from the union to the actual target operator. - * - * @throws CompilerException - */ - private DistributionPattern connectJobVertices( - Channel channel, - int inputNumber, - final JobVertex sourceVertex, - final TaskConfig sourceConfig, - final JobVertex targetVertex, - final TaskConfig targetConfig, - boolean isBroadcast) - throws CompilerException { - // ------------ connect the vertices to the job graph -------------- - final DistributionPattern distributionPattern; - - switch (channel.getShipStrategy()) { - case FORWARD: - distributionPattern = DistributionPattern.POINTWISE; - break; - case PARTITION_RANDOM: - case BROADCAST: - case PARTITION_HASH: - case PARTITION_CUSTOM: - case PARTITION_RANGE: - case PARTITION_FORCED_REBALANCE: - distributionPattern = DistributionPattern.ALL_TO_ALL; - break; - default: - throw new RuntimeException( - "Unknown runtime ship strategy: " + channel.getShipStrategy()); - } - - final ResultPartitionType resultType; - - switch (channel.getDataExchangeMode()) { - case PIPELINED: - resultType = ResultPartitionType.PIPELINED; - break; - - case BATCH: - // BLOCKING results are currently not supported in closed loop iterations - // - // See https://issues.apache.org/jira/browse/FLINK-1713 for details - resultType = - channel.getSource().isOnDynamicPath() - ? ResultPartitionType.PIPELINED - : ResultPartitionType.BLOCKING; - break; - - case PIPELINE_WITH_BATCH_FALLBACK: - throw new UnsupportedOperationException( - "Data exchange mode " - + channel.getDataExchangeMode() - + " currently not supported."); - - default: - throw new UnsupportedOperationException("Unknown data exchange mode."); - } - - JobEdge edge = - targetVertex.connectNewDataSetAsInput( - sourceVertex, distributionPattern, resultType, isBroadcast); - - // -------------- configure the source task's ship strategy strategies in task config - // -------------- - final int outputIndex = sourceConfig.getNumOutputs(); - sourceConfig.addOutputShipStrategy(channel.getShipStrategy()); - if (outputIndex == 0) { - sourceConfig.setOutputSerializer(channel.getSerializer()); - } - if (channel.getShipStrategyComparator() != null) { - sourceConfig.setOutputComparator(channel.getShipStrategyComparator(), outputIndex); - } - - if (channel.getShipStrategy() == ShipStrategyType.PARTITION_RANGE) { - - final DataDistribution dataDistribution = channel.getDataDistribution(); - if (dataDistribution != null) { - sourceConfig.setOutputDataDistribution(dataDistribution, outputIndex); - } else { - throw new RuntimeException("Range partitioning requires data distribution."); - } - } - - if (channel.getShipStrategy() == ShipStrategyType.PARTITION_CUSTOM) { - if (channel.getPartitioner() != null) { - sourceConfig.setOutputPartitioner(channel.getPartitioner(), outputIndex); - } else { - throw new CompilerException( - "The ship strategy was set to custom partitioning, but no partitioner was set."); - } - } - - // ---------------- configure the receiver ------------------- - if (isBroadcast) { - targetConfig.addBroadcastInputToGroup(inputNumber); - } else { - targetConfig.addInputToGroup(inputNumber); - } - - // ---------------- attach the additional infos to the job edge ------------------- - - String shipStrategy = JsonMapper.getShipStrategyString(channel.getShipStrategy()); - if (channel.getShipStrategyKeys() != null && channel.getShipStrategyKeys().size() > 0) { - shipStrategy += - " on " - + (channel.getShipStrategySortOrder() == null - ? channel.getShipStrategyKeys().toString() - : Utils.createOrdering( - channel.getShipStrategyKeys(), - channel.getShipStrategySortOrder()) - .toString()); - } - - String localStrategy; - if (channel.getLocalStrategy() == null - || channel.getLocalStrategy() == LocalStrategy.NONE) { - localStrategy = null; - } else { - localStrategy = JsonMapper.getLocalStrategyString(channel.getLocalStrategy()); - if (localStrategy != null - && channel.getLocalStrategyKeys() != null - && channel.getLocalStrategyKeys().size() > 0) { - localStrategy += - " on " - + (channel.getLocalStrategySortOrder() == null - ? channel.getLocalStrategyKeys().toString() - : Utils.createOrdering( - channel.getLocalStrategyKeys(), - channel.getLocalStrategySortOrder()) - .toString()); - } - } - - String caching = - channel.getTempMode() == TempMode.NONE ? null : channel.getTempMode().toString(); - - edge.setShipStrategyName(shipStrategy); - edge.setForward(channel.getShipStrategy() == ShipStrategyType.FORWARD); - edge.setPreProcessingOperationName(localStrategy); - edge.setOperatorLevelCachingDescription(caching); - - return distributionPattern; - } - - private void addLocalInfoFromChannelToConfig( - Channel channel, TaskConfig config, int inputNum, boolean isBroadcastChannel) { - // serializer - if (isBroadcastChannel) { - config.setBroadcastInputSerializer(channel.getSerializer(), inputNum); - - if (channel.getLocalStrategy() != LocalStrategy.NONE - || (channel.getTempMode() != null && channel.getTempMode() != TempMode.NONE)) { - throw new CompilerException( - "Found local strategy or temp mode on a broadcast variable channel."); - } else { - return; - } - } else { - config.setInputSerializer(channel.getSerializer(), inputNum); - } - - // local strategy - if (channel.getLocalStrategy() != LocalStrategy.NONE) { - config.setInputLocalStrategy(inputNum, channel.getLocalStrategy()); - if (channel.getLocalStrategyComparator() != null) { - config.setInputComparator(channel.getLocalStrategyComparator(), inputNum); - } - } - - assignLocalStrategyResources(channel, config, inputNum); - - // materialization / caching - if (channel.getTempMode() != null) { - final TempMode tm = channel.getTempMode(); - - boolean needsMemory = false; - // Don't add a pipeline breaker if the data exchange is already blocking, EXCEPT the - // channel is within an iteration. - if (tm.breaksPipeline() - && (channel.isOnDynamicPath() - || channel.getDataExchangeMode() != DataExchangeMode.BATCH)) { - config.setInputAsynchronouslyMaterialized(inputNum, true); - needsMemory = true; - } - if (tm.isCached()) { - config.setInputCached(inputNum, true); - needsMemory = true; - } - - if (needsMemory) { - // sanity check - if (tm == TempMode.NONE || channel.getRelativeTempMemory() <= 0) { - throw new CompilerException( - "Bug in compiler: Inconsistent description of input materialization."); - } - config.setRelativeInputMaterializationMemory( - inputNum, channel.getRelativeTempMemory()); - } - } - } - - private void finalizeBulkIteration(IterationDescriptor descr) { - - final BulkIterationPlanNode bulkNode = (BulkIterationPlanNode) descr.getIterationNode(); - final JobVertex headVertex = descr.getHeadTask(); - final TaskConfig headConfig = new TaskConfig(headVertex.getConfiguration()); - final TaskConfig headFinalOutputConfig = descr.getHeadFinalResultConfig(); - - // ------------ finalize the head config with the final outputs and the sync gate - // ------------ - final int numStepFunctionOuts = headConfig.getNumOutputs(); - final int numFinalOuts = headFinalOutputConfig.getNumOutputs(); - - if (numStepFunctionOuts == 0) { - throw new CompilerException("The iteration has no operation inside the step function."); - } - - headConfig.setIterationHeadFinalOutputConfig(headFinalOutputConfig); - headConfig.setIterationHeadIndexOfSyncOutput(numStepFunctionOuts + numFinalOuts); - final double relativeMemForBackChannel = bulkNode.getRelativeMemoryPerSubTask(); - if (relativeMemForBackChannel <= 0) { - throw new CompilerException( - "Bug: No memory has been assigned to the iteration back channel."); - } - headConfig.setRelativeBackChannelMemory(relativeMemForBackChannel); - - // --------------------------- create the sync task --------------------------- - final JobVertex sync = new JobVertex("Sync (" + bulkNode.getNodeName() + ")"); - sync.setResources(bulkNode.getMinResources(), bulkNode.getPreferredResources()); - sync.setInvokableClass(IterationSynchronizationSinkTask.class); - sync.setParallelism(1); - sync.setMaxParallelism(1); - this.auxVertices.add(sync); - - final TaskConfig syncConfig = new TaskConfig(sync.getConfiguration()); - syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, headVertex.getParallelism()); - - // set the number of iteration / convergence criterion for the sync - final int maxNumIterations = - bulkNode.getIterationNode().getIterationContract().getMaximumNumberOfIterations(); - if (maxNumIterations < 1) { - throw new CompilerException( - "Cannot create bulk iteration with unspecified maximum number of iterations."); - } - syncConfig.setNumberOfIterations(maxNumIterations); - - // connect the sync task - sync.connectNewDataSetAsInput( - headVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - - // ----------------------------- create the iteration tail ------------------------------ - - final PlanNode rootOfTerminationCriterion = bulkNode.getRootOfTerminationCriterion(); - final PlanNode rootOfStepFunction = bulkNode.getRootOfStepFunction(); - final TaskConfig tailConfig; - - JobVertex rootOfStepFunctionVertex = this.vertices.get(rootOfStepFunction); - if (rootOfStepFunctionVertex == null) { - // last op is chained - final TaskInChain taskInChain = this.chainedTasks.get(rootOfStepFunction); - if (taskInChain == null) { - throw new CompilerException( - "Bug: Tail of step function not found as vertex or chained task."); - } - rootOfStepFunctionVertex = taskInChain.getContainingVertex(); - - // the fake channel is statically typed to pact record. no data is sent over this - // channel anyways. - tailConfig = taskInChain.getTaskConfig(); - } else { - tailConfig = new TaskConfig(rootOfStepFunctionVertex.getConfiguration()); - } - - tailConfig.setIsWorksetUpdate(); - - // No following termination criterion - if (rootOfStepFunction.getOutgoingChannels().isEmpty()) { - - rootOfStepFunctionVertex.setInvokableClass(IterationTailTask.class); - - tailConfig.setOutputSerializer(bulkNode.getSerializerForIterationChannel()); - } - - // create the fake output task for termination criterion, if needed - final TaskConfig tailConfigOfTerminationCriterion; - // If we have a termination criterion and it is not an intermediate node - if (rootOfTerminationCriterion != null - && rootOfTerminationCriterion.getOutgoingChannels().isEmpty()) { - JobVertex rootOfTerminationCriterionVertex = - this.vertices.get(rootOfTerminationCriterion); - - if (rootOfTerminationCriterionVertex == null) { - // last op is chained - final TaskInChain taskInChain = this.chainedTasks.get(rootOfTerminationCriterion); - if (taskInChain == null) { - throw new CompilerException( - "Bug: Tail of termination criterion not found as vertex or chained task."); - } - rootOfTerminationCriterionVertex = taskInChain.getContainingVertex(); - - // the fake channel is statically typed to pact record. no data is sent over this - // channel anyways. - tailConfigOfTerminationCriterion = taskInChain.getTaskConfig(); - } else { - tailConfigOfTerminationCriterion = - new TaskConfig(rootOfTerminationCriterionVertex.getConfiguration()); - } - - rootOfTerminationCriterionVertex.setInvokableClass(IterationTailTask.class); - // Hack - tailConfigOfTerminationCriterion.setIsSolutionSetUpdate(); - tailConfigOfTerminationCriterion.setOutputSerializer( - bulkNode.getSerializerForIterationChannel()); - - // tell the head that it needs to wait for the solution set updates - headConfig.setWaitForSolutionSetUpdate(); - } - - // ------------------- register the aggregators ------------------- - AggregatorRegistry aggs = - bulkNode.getIterationNode().getIterationContract().getAggregators(); - Collection> allAggregators = aggs.getAllRegisteredAggregators(); - - headConfig.addIterationAggregators(allAggregators); - syncConfig.addIterationAggregators(allAggregators); - - String convAggName = aggs.getConvergenceCriterionAggregatorName(); - ConvergenceCriterion convCriterion = aggs.getConvergenceCriterion(); - - if (convCriterion != null || convAggName != null) { - if (convCriterion == null) { - throw new CompilerException( - "Error: Convergence criterion aggregator set, but criterion is null."); - } - if (convAggName == null) { - throw new CompilerException( - "Error: Aggregator convergence criterion set, but aggregator is null."); - } - - syncConfig.setConvergenceCriterion(convAggName, convCriterion); - } - } - - private void finalizeWorksetIteration(IterationDescriptor descr) { - final WorksetIterationPlanNode iterNode = - (WorksetIterationPlanNode) descr.getIterationNode(); - final JobVertex headVertex = descr.getHeadTask(); - final TaskConfig headConfig = new TaskConfig(headVertex.getConfiguration()); - final TaskConfig headFinalOutputConfig = descr.getHeadFinalResultConfig(); - - // ------------ finalize the head config with the final outputs and the sync gate - // ------------ - { - final int numStepFunctionOuts = headConfig.getNumOutputs(); - final int numFinalOuts = headFinalOutputConfig.getNumOutputs(); - - if (numStepFunctionOuts == 0) { - throw new CompilerException( - "The workset iteration has no operation on the workset inside the step function."); - } - - headConfig.setIterationHeadFinalOutputConfig(headFinalOutputConfig); - headConfig.setIterationHeadIndexOfSyncOutput(numStepFunctionOuts + numFinalOuts); - final double relativeMemory = iterNode.getRelativeMemoryPerSubTask(); - if (relativeMemory <= 0) { - throw new CompilerException( - "Bug: No memory has been assigned to the workset iteration."); - } - - headConfig.setIsWorksetIteration(); - headConfig.setRelativeBackChannelMemory(relativeMemory / 2); - headConfig.setRelativeSolutionSetMemory(relativeMemory / 2); - - // set the solution set serializer and comparator - headConfig.setSolutionSetSerializer(iterNode.getSolutionSetSerializer()); - headConfig.setSolutionSetComparator(iterNode.getSolutionSetComparator()); - } - - // --------------------------- create the sync task --------------------------- - final TaskConfig syncConfig; - { - final JobVertex sync = new JobVertex("Sync (" + iterNode.getNodeName() + ")"); - sync.setResources(iterNode.getMinResources(), iterNode.getPreferredResources()); - sync.setInvokableClass(IterationSynchronizationSinkTask.class); - sync.setParallelism(1); - sync.setMaxParallelism(1); - this.auxVertices.add(sync); - - syncConfig = new TaskConfig(sync.getConfiguration()); - syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt( - 0, headVertex.getParallelism()); - - // set the number of iteration / convergence criterion for the sync - final int maxNumIterations = - iterNode.getIterationNode() - .getIterationContract() - .getMaximumNumberOfIterations(); - if (maxNumIterations < 1) { - throw new CompilerException( - "Cannot create workset iteration with unspecified maximum number of iterations."); - } - syncConfig.setNumberOfIterations(maxNumIterations); - - // connect the sync task - sync.connectNewDataSetAsInput( - headVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - } - - // ----------------------------- create the iteration tails ----------------------------- - // ----------------------- for next workset and solution set delta----------------------- - - { - // we have three possible cases: - // 1) Two tails, one for workset update, one for solution set update - // 2) One tail for workset update, solution set update happens in an intermediate task - // 3) One tail for solution set update, workset update happens in an intermediate task - - final PlanNode nextWorksetNode = iterNode.getNextWorkSetPlanNode(); - final PlanNode solutionDeltaNode = iterNode.getSolutionSetDeltaPlanNode(); - - final boolean hasWorksetTail = nextWorksetNode.getOutgoingChannels().isEmpty(); - final boolean hasSolutionSetTail = - (!iterNode.isImmediateSolutionSetUpdate()) || (!hasWorksetTail); - - { - // get the vertex for the workset update - final TaskConfig worksetTailConfig; - JobVertex nextWorksetVertex = this.vertices.get(nextWorksetNode); - if (nextWorksetVertex == null) { - // nextWorksetVertex is chained - TaskInChain taskInChain = this.chainedTasks.get(nextWorksetNode); - if (taskInChain == null) { - throw new CompilerException( - "Bug: Next workset node not found as vertex or chained task."); - } - nextWorksetVertex = taskInChain.getContainingVertex(); - worksetTailConfig = taskInChain.getTaskConfig(); - } else { - worksetTailConfig = new TaskConfig(nextWorksetVertex.getConfiguration()); - } - - // mark the node to perform workset updates - worksetTailConfig.setIsWorksetIteration(); - worksetTailConfig.setIsWorksetUpdate(); - - if (hasWorksetTail) { - nextWorksetVertex.setInvokableClass(IterationTailTask.class); - - worksetTailConfig.setOutputSerializer(iterNode.getWorksetSerializer()); - } - } - { - final TaskConfig solutionDeltaConfig; - JobVertex solutionDeltaVertex = this.vertices.get(solutionDeltaNode); - if (solutionDeltaVertex == null) { - // last op is chained - TaskInChain taskInChain = this.chainedTasks.get(solutionDeltaNode); - if (taskInChain == null) { - throw new CompilerException( - "Bug: Solution Set Delta not found as vertex or chained task."); - } - solutionDeltaVertex = taskInChain.getContainingVertex(); - solutionDeltaConfig = taskInChain.getTaskConfig(); - } else { - solutionDeltaConfig = new TaskConfig(solutionDeltaVertex.getConfiguration()); - } - - solutionDeltaConfig.setIsWorksetIteration(); - solutionDeltaConfig.setIsSolutionSetUpdate(); - - if (hasSolutionSetTail) { - solutionDeltaVertex.setInvokableClass(IterationTailTask.class); - - solutionDeltaConfig.setOutputSerializer(iterNode.getSolutionSetSerializer()); - - // tell the head that it needs to wait for the solution set updates - headConfig.setWaitForSolutionSetUpdate(); - } else { - // no tail, intermediate update. must be immediate update - if (!iterNode.isImmediateSolutionSetUpdate()) { - throw new CompilerException( - "A solution set update without dedicated tail is not set to perform immediate updates."); - } - solutionDeltaConfig.setIsSolutionSetUpdateWithoutReprobe(); - } - } - } - - // ------------------- register the aggregators ------------------- - AggregatorRegistry aggs = - iterNode.getIterationNode().getIterationContract().getAggregators(); - Collection> allAggregators = aggs.getAllRegisteredAggregators(); - - for (AggregatorWithName agg : allAggregators) { - if (agg.getName().equals(WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME)) { - throw new CompilerException( - "User defined aggregator used the same name as built-in workset " - + "termination check aggregator: " - + WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME); - } - } - - headConfig.addIterationAggregators(allAggregators); - syncConfig.addIterationAggregators(allAggregators); - - String convAggName = aggs.getConvergenceCriterionAggregatorName(); - ConvergenceCriterion convCriterion = aggs.getConvergenceCriterion(); - - if (convCriterion != null || convAggName != null) { - if (convCriterion == null) { - throw new CompilerException( - "Error: Convergence criterion aggregator set, but criterion is null."); - } - if (convAggName == null) { - throw new CompilerException( - "Error: Aggregator convergence criterion set, but aggregator is null."); - } - - syncConfig.setConvergenceCriterion(convAggName, convCriterion); - } - - headConfig.addIterationAggregator( - WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME, new LongSumAggregator()); - syncConfig.addIterationAggregator( - WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME, new LongSumAggregator()); - syncConfig.setImplicitConvergenceCriterion( - WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME, - new WorksetEmptyConvergenceCriterion()); - } - - private String getDescriptionForUserCode(UserCodeWrapper wrapper) { - try { - if (wrapper.hasObject()) { - try { - return wrapper.getUserCodeObject().toString(); - } catch (Throwable t) { - return wrapper.getUserCodeClass().getName(); - } - } else { - return wrapper.getUserCodeClass().getName(); - } - } catch (Throwable t) { - return null; - } - } - - private void attachOperatorNamesAndDescriptions() { - JsonFactory jsonFactory = new JsonFactory(); - - // we go back to front - - // start with the in chains - for (int i = chainedTasksInSequence.size() - 1; i >= 0; i--) { - TaskInChain next = chainedTasksInSequence.get(i); - PlanNode planNode = next.getPlanNode(); - - JobVertex vertex = next.getContainingVertex(); - - // operator - String opName = planNode.getOptimizerNode().getOperatorName(); - if (vertex.getOperatorName() == null) { - vertex.setOperatorName(opName); - } else { - vertex.setOperatorName(opName + " -> " + vertex.getOperatorName()); - } - - // operator description - String opDescription = - JsonMapper.getOperatorStrategyString(planNode.getDriverStrategy()); - if (vertex.getOperatorDescription() == null) { - vertex.setOperatorDescription(opDescription); - } else { - vertex.setOperatorDescription( - opDescription + "\n -> " + vertex.getOperatorDescription()); - } - - // pretty name - String prettyName = StringUtils.showControlCharacters(planNode.getNodeName()); - if (vertex.getOperatorPrettyName() == null) { - vertex.setOperatorPrettyName(prettyName); - } else { - vertex.setOperatorPrettyName( - prettyName + "\n -> " + vertex.getOperatorPrettyName()); - } - - // optimizer output properties - if (vertex.getResultOptimizerProperties() == null) { - // since we go backwards, this must be the last in its chain - String outputProps = JsonMapper.getOptimizerPropertiesJson(jsonFactory, planNode); - vertex.setResultOptimizerProperties(outputProps); - } - } - - // finish back-to-front traversal by going over the head vertices - for (Map.Entry entry : vertices.entrySet()) { - PlanNode node = entry.getKey(); - JobVertex vertex = entry.getValue(); - - // get the predecessors - - String input1name = null; - String input2name = null; - int num = 0; - for (Channel c : node.getInputs()) { - if (num == 0) { - input1name = c.getSource().getNodeName(); - } else if (num == 1) { - input2name = c.getSource().getNodeName(); - } - num++; - } - - // operator - String opName = node.getOptimizerNode().getOperatorName(); - if (vertex.getOperatorName() == null) { - vertex.setOperatorName(opName); - } else { - vertex.setOperatorName(opName + " -> " + vertex.getOperatorName()); - } - - // operator description - String opStrategy = - JsonMapper.getOperatorStrategyString( - node.getDriverStrategy(), - input1name != null ? input1name : "(unnamed)", - input2name != null ? input2name : "(unnamed)"); - - if (vertex.getOperatorDescription() == null) { - vertex.setOperatorDescription(opStrategy); - } else { - vertex.setOperatorDescription( - opStrategy + "\n -> " + vertex.getOperatorDescription()); - } - - // pretty name - String prettyName = StringUtils.showControlCharacters(node.getNodeName()); - if (vertex.getOperatorPrettyName() == null) { - vertex.setOperatorPrettyName(prettyName); - } else { - vertex.setOperatorPrettyName( - prettyName + "\n -> " + vertex.getOperatorPrettyName()); - } - - // if there is not yet an output from a chained task, we set this output - if (vertex.getResultOptimizerProperties() == null) { - vertex.setResultOptimizerProperties( - JsonMapper.getOptimizerPropertiesJson(jsonFactory, node)); - } - } - } - - // ------------------------------------------------------------------------------------- - // Descriptors for tasks / configurations that are chained or merged with other tasks - // ------------------------------------------------------------------------------------- - - /** - * Utility class that describes a task in a sequence of chained tasks. Chained tasks are tasks - * that run together in one thread. - */ - private static final class TaskInChain { - - private final Class> chainedTask; - - private final TaskConfig taskConfig; - - private final String taskName; - - private final PlanNode planNode; - - private JobVertex containingVertex; - - TaskInChain( - PlanNode planNode, - Class> chainedTask, - TaskConfig taskConfig, - String taskName) { - - this.planNode = planNode; - this.chainedTask = chainedTask; - this.taskConfig = taskConfig; - this.taskName = taskName; - } - - public PlanNode getPlanNode() { - return planNode; - } - - public Class> getChainedTask() { - return this.chainedTask; - } - - public TaskConfig getTaskConfig() { - return this.taskConfig; - } - - public String getTaskName() { - return this.taskName; - } - - public JobVertex getContainingVertex() { - return this.containingVertex; - } - - public void setContainingVertex(JobVertex containingVertex) { - this.containingVertex = containingVertex; - } - } - - private static final class IterationDescriptor { - - private final IterationPlanNode iterationNode; - - private JobVertex headTask; - - private TaskConfig headConfig; - - private TaskConfig headFinalResultConfig; - - private final int id; - - public IterationDescriptor(IterationPlanNode iterationNode, int id) { - this.iterationNode = iterationNode; - this.id = id; - } - - public IterationPlanNode getIterationNode() { - return iterationNode; - } - - public void setHeadTask(JobVertex headTask, TaskConfig headConfig) { - this.headTask = headTask; - this.headFinalResultConfig = new TaskConfig(new Configuration()); - - // check if we already had a configuration, for example if the solution set was - if (this.headConfig != null) { - headConfig.getConfiguration().addAll(this.headConfig.getConfiguration()); - } - - this.headConfig = headConfig; - } - - public JobVertex getHeadTask() { - return headTask; - } - - public TaskConfig getHeadFinalResultConfig() { - return headFinalResultConfig; - } - - public int getId() { - return this.id; - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JsonMapper.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JsonMapper.java deleted file mode 100644 index 6046c533b9f8c..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JsonMapper.java +++ /dev/null @@ -1,339 +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.optimizer.plantranslate; - -import org.apache.flink.api.common.operators.CompilerHints; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; - -import java.io.IOException; -import java.io.StringWriter; - -import static org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator.formatNumber; - -public class JsonMapper { - - public static String getOperatorStrategyString(DriverStrategy strategy) { - return getOperatorStrategyString(strategy, "input 1", "input 2"); - } - - public static String getOperatorStrategyString( - DriverStrategy strategy, String firstInputName, String secondInputName) { - if (strategy == null) { - return "(null)"; - } - switch (strategy) { - case SOURCE: - return "Data Source"; - case SINK: - return "Data Sink"; - - case NONE: - return "(none)"; - - case BINARY_NO_OP: - case UNARY_NO_OP: - return "No-Op"; - - case MAP: - return "Map"; - - case FLAT_MAP: - return "FlatMap"; - - case MAP_PARTITION: - return "Map Partition"; - - case ALL_REDUCE: - return "Reduce All"; - - case ALL_GROUP_REDUCE: - case ALL_GROUP_REDUCE_COMBINE: - return "Group Reduce All"; - - case SORTED_REDUCE: - return "Sorted Reduce"; - - case SORTED_PARTIAL_REDUCE: - return "Sorted Combine/Reduce"; - - case SORTED_GROUP_REDUCE: - return "Sorted Group Reduce"; - - case SORTED_GROUP_COMBINE: - return "Sorted Combine"; - - case HYBRIDHASH_BUILD_FIRST: - return "Hybrid Hash (build: " + firstInputName + ")"; - - case HYBRIDHASH_BUILD_SECOND: - return "Hybrid Hash (build: " + secondInputName + ")"; - - case HYBRIDHASH_BUILD_FIRST_CACHED: - return "Hybrid Hash (CACHED) (build: " + firstInputName + ")"; - - case HYBRIDHASH_BUILD_SECOND_CACHED: - return "Hybrid Hash (CACHED) (build: " + secondInputName + ")"; - - case NESTEDLOOP_BLOCKED_OUTER_FIRST: - return "Nested Loops (Blocked Outer: " + firstInputName + ")"; - case NESTEDLOOP_BLOCKED_OUTER_SECOND: - return "Nested Loops (Blocked Outer: " + secondInputName + ")"; - case NESTEDLOOP_STREAMED_OUTER_FIRST: - return "Nested Loops (Streamed Outer: " + firstInputName + ")"; - case NESTEDLOOP_STREAMED_OUTER_SECOND: - return "Nested Loops (Streamed Outer: " + secondInputName + ")"; - - case INNER_MERGE: - return "Merge"; - case FULL_OUTER_MERGE: - return "Full Outer Merge"; - case LEFT_OUTER_MERGE: - return "Left Outer Merge"; - case RIGHT_OUTER_MERGE: - return "Right Outer Merge"; - - case CO_GROUP: - return "Co-Group"; - - default: - return strategy.name(); - } - } - - public static String getShipStrategyString(ShipStrategyType shipType) { - if (shipType == null) { - return "(null)"; - } - switch (shipType) { - case NONE: - return "(none)"; - case FORWARD: - return "Forward"; - case BROADCAST: - return "Broadcast"; - case PARTITION_HASH: - return "Hash Partition"; - case PARTITION_RANGE: - return "Range Partition"; - case PARTITION_RANDOM: - return "Redistribute"; - case PARTITION_FORCED_REBALANCE: - return "Rebalance"; - case PARTITION_CUSTOM: - return "Custom Partition"; - default: - return shipType.name(); - } - } - - public static String getLocalStrategyString(LocalStrategy localStrategy) { - if (localStrategy == null) { - return "(null)"; - } - switch (localStrategy) { - case NONE: - return "(none)"; - case SORT: - return "Sort"; - case COMBININGSORT: - return "Sort (combining)"; - default: - return localStrategy.name(); - } - } - - public static String getOptimizerPropertiesJson(JsonFactory jsonFactory, PlanNode node) { - try { - final StringWriter writer = new StringWriter(256); - final JsonGenerator gen = jsonFactory.createGenerator(writer); - - final OptimizerNode optNode = node.getOptimizerNode(); - - gen.writeStartObject(); - - // global properties - if (node.getGlobalProperties() != null) { - GlobalProperties gp = node.getGlobalProperties(); - gen.writeArrayFieldStart("global_properties"); - - addProperty(gen, "Partitioning", gp.getPartitioning().name()); - if (gp.getPartitioningFields() != null) { - addProperty(gen, "Partitioned on", gp.getPartitioningFields().toString()); - } - if (gp.getPartitioningOrdering() != null) { - addProperty(gen, "Partitioning Order", gp.getPartitioningOrdering().toString()); - } else { - addProperty(gen, "Partitioning Order", "(none)"); - } - if (optNode.getUniqueFields() == null || optNode.getUniqueFields().size() == 0) { - addProperty(gen, "Uniqueness", "not unique"); - } else { - addProperty(gen, "Uniqueness", optNode.getUniqueFields().toString()); - } - - gen.writeEndArray(); - } - - // local properties - if (node.getLocalProperties() != null) { - LocalProperties lp = node.getLocalProperties(); - gen.writeArrayFieldStart("local_properties"); - - if (lp.getOrdering() != null) { - addProperty(gen, "Order", lp.getOrdering().toString()); - } else { - addProperty(gen, "Order", "(none)"); - } - if (lp.getGroupedFields() != null && lp.getGroupedFields().size() > 0) { - addProperty(gen, "Grouped on", lp.getGroupedFields().toString()); - } else { - addProperty(gen, "Grouping", "not grouped"); - } - if (optNode.getUniqueFields() == null || optNode.getUniqueFields().size() == 0) { - addProperty(gen, "Uniqueness", "not unique"); - } else { - addProperty(gen, "Uniqueness", optNode.getUniqueFields().toString()); - } - - gen.writeEndArray(); - } - - // output size estimates - { - gen.writeArrayFieldStart("estimates"); - - addProperty( - gen, - "Est. Output Size", - optNode.getEstimatedOutputSize() == -1 - ? "(unknown)" - : formatNumber(optNode.getEstimatedOutputSize(), "B")); - - addProperty( - gen, - "Est. Cardinality", - optNode.getEstimatedNumRecords() == -1 - ? "(unknown)" - : formatNumber(optNode.getEstimatedNumRecords())); - gen.writeEndArray(); - } - - // output node cost - if (node.getNodeCosts() != null) { - gen.writeArrayFieldStart("costs"); - - addProperty( - gen, - "Network", - node.getNodeCosts().getNetworkCost() == -1 - ? "(unknown)" - : formatNumber(node.getNodeCosts().getNetworkCost(), "B")); - addProperty( - gen, - "Disk I/O", - node.getNodeCosts().getDiskCost() == -1 - ? "(unknown)" - : formatNumber(node.getNodeCosts().getDiskCost(), "B")); - addProperty( - gen, - "CPU", - node.getNodeCosts().getCpuCost() == -1 - ? "(unknown)" - : formatNumber(node.getNodeCosts().getCpuCost(), "")); - - addProperty( - gen, - "Cumulative Network", - node.getCumulativeCosts().getNetworkCost() == -1 - ? "(unknown)" - : formatNumber(node.getCumulativeCosts().getNetworkCost(), "B")); - addProperty( - gen, - "Cumulative Disk I/O", - node.getCumulativeCosts().getDiskCost() == -1 - ? "(unknown)" - : formatNumber(node.getCumulativeCosts().getDiskCost(), "B")); - addProperty( - gen, - "Cumulative CPU", - node.getCumulativeCosts().getCpuCost() == -1 - ? "(unknown)" - : formatNumber(node.getCumulativeCosts().getCpuCost(), "")); - - gen.writeEndArray(); - } - - // compiler hints - if (optNode.getOperator().getCompilerHints() != null) { - CompilerHints hints = optNode.getOperator().getCompilerHints(); - CompilerHints defaults = new CompilerHints(); - - String size = - hints.getOutputSize() == defaults.getOutputSize() - ? "(none)" - : String.valueOf(hints.getOutputSize()); - String card = - hints.getOutputCardinality() == defaults.getOutputCardinality() - ? "(none)" - : String.valueOf(hints.getOutputCardinality()); - String width = - hints.getAvgOutputRecordSize() == defaults.getAvgOutputRecordSize() - ? "(none)" - : String.valueOf(hints.getAvgOutputRecordSize()); - String filter = - hints.getFilterFactor() == defaults.getFilterFactor() - ? "(none)" - : String.valueOf(hints.getFilterFactor()); - - gen.writeArrayFieldStart("compiler_hints"); - - addProperty(gen, "Output Size (bytes)", size); - addProperty(gen, "Output Cardinality", card); - addProperty(gen, "Avg. Output Record Size (bytes)", width); - addProperty(gen, "Filter Factor", filter); - - gen.writeEndArray(); - } - - gen.writeEndObject(); - - gen.close(); - return writer.toString(); - } catch (Exception e) { - return "{}"; - } - } - - private static void addProperty(JsonGenerator gen, String name, String value) - throws IOException { - gen.writeStartObject(); - gen.writeStringField("name", name); - gen.writeStringField("value", value); - gen.writeEndObject(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/ConflictingFieldTypeInfoException.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/ConflictingFieldTypeInfoException.java deleted file mode 100644 index 255b0e2cf48f7..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/ConflictingFieldTypeInfoException.java +++ /dev/null @@ -1,54 +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.optimizer.postpass; - -public final class ConflictingFieldTypeInfoException extends Exception { - - private static final long serialVersionUID = 3991352502693288321L; - - private final int fieldNumber; - - private final Object previousType, newType; - - public ConflictingFieldTypeInfoException(int fieldNumber, Object previousType, Object newType) { - super( - "Conflicting type info for field " - + fieldNumber - + ": Old='" - + previousType - + "', new='" - + newType - + "'."); - this.fieldNumber = fieldNumber; - this.previousType = previousType; - this.newType = newType; - } - - public int getFieldNumber() { - return fieldNumber; - } - - public Object getPreviousType() { - return this.previousType; - } - - public Object getNewType() { - return this.newType; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/JavaApiPostPass.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/JavaApiPostPass.java deleted file mode 100644 index 476762243186e..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/JavaApiPostPass.java +++ /dev/null @@ -1,353 +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.optimizer.postpass; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.operators.DualInputOperator; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.base.BulkIterationBase; -import org.apache.flink.api.common.operators.base.DeltaIterationBase; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeinfo.AtomicType; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.CompositeType; -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.typeutils.runtime.RuntimeComparatorFactory; -import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory; -import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.CompilerPostPassException; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SolutionSetPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plan.WorksetPlanNode; -import org.apache.flink.optimizer.util.NoOpUnaryUdfOp; -import org.apache.flink.runtime.operators.DriverStrategy; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; - -/** - * The post-optimizer plan traversal. This traversal fills in the API specific utilities - * (serializers and comparators). - */ -public class JavaApiPostPass implements OptimizerPostPass { - - private final Set alreadyDone = new HashSet(); - - private ExecutionConfig executionConfig = null; - - @Override - public void postPass(OptimizedPlan plan) { - - executionConfig = plan.getOriginalPlan().getExecutionConfig(); - - for (SinkPlanNode sink : plan.getDataSinks()) { - traverse(sink); - } - } - - protected void traverse(PlanNode node) { - if (!alreadyDone.add(node)) { - // already worked on that one - return; - } - - // distinguish the node types - if (node instanceof SinkPlanNode) { - // descend to the input channel - SinkPlanNode sn = (SinkPlanNode) node; - Channel inchannel = sn.getInput(); - traverseChannel(inchannel); - } else if (node instanceof SourcePlanNode) { - TypeInformation typeInfo = getTypeInfoFromSource((SourcePlanNode) node); - ((SourcePlanNode) node).setSerializer(createSerializer(typeInfo)); - } else if (node instanceof BulkIterationPlanNode) { - BulkIterationPlanNode iterationNode = (BulkIterationPlanNode) node; - - if (iterationNode.getRootOfStepFunction() instanceof NAryUnionPlanNode) { - throw new CompilerException( - "Optimizer cannot compile an iteration step function where next partial solution is created by a Union node."); - } - - // traverse the termination criterion for the first time. create schema only, no - // utilities. Needed in case of intermediate termination criterion - if (iterationNode.getRootOfTerminationCriterion() != null) { - SingleInputPlanNode addMapper = - (SingleInputPlanNode) iterationNode.getRootOfTerminationCriterion(); - traverseChannel(addMapper.getInput()); - } - - BulkIterationBase operator = - (BulkIterationBase) iterationNode.getProgramOperator(); - - // set the serializer - iterationNode.setSerializerForIterationChannel( - createSerializer(operator.getOperatorInfo().getOutputType())); - - // done, we can now propagate our info down - traverseChannel(iterationNode.getInput()); - traverse(iterationNode.getRootOfStepFunction()); - } else if (node instanceof WorksetIterationPlanNode) { - WorksetIterationPlanNode iterationNode = (WorksetIterationPlanNode) node; - - if (iterationNode.getNextWorkSetPlanNode() instanceof NAryUnionPlanNode) { - throw new CompilerException( - "Optimizer cannot compile a workset iteration step function where the next workset is produced by a Union node."); - } - if (iterationNode.getSolutionSetDeltaPlanNode() instanceof NAryUnionPlanNode) { - throw new CompilerException( - "Optimizer cannot compile a workset iteration step function where the solution set delta is produced by a Union node."); - } - - DeltaIterationBase operator = - (DeltaIterationBase) iterationNode.getProgramOperator(); - - // set the serializers and comparators for the workset iteration - iterationNode.setSolutionSetSerializer( - createSerializer(operator.getOperatorInfo().getFirstInputType())); - iterationNode.setWorksetSerializer( - createSerializer(operator.getOperatorInfo().getSecondInputType())); - iterationNode.setSolutionSetComparator( - createComparator( - operator.getOperatorInfo().getFirstInputType(), - iterationNode.getSolutionSetKeyFields(), - getSortOrders(iterationNode.getSolutionSetKeyFields(), null))); - - // traverse the inputs - traverseChannel(iterationNode.getInput1()); - traverseChannel(iterationNode.getInput2()); - - // traverse the step function - traverse(iterationNode.getSolutionSetDeltaPlanNode()); - traverse(iterationNode.getNextWorkSetPlanNode()); - } else if (node instanceof SingleInputPlanNode) { - SingleInputPlanNode sn = (SingleInputPlanNode) node; - - if (!(sn.getOptimizerNode().getOperator() instanceof SingleInputOperator)) { - - // Special case for delta iterations - if (sn.getOptimizerNode().getOperator() instanceof NoOpUnaryUdfOp) { - traverseChannel(sn.getInput()); - return; - } else { - throw new RuntimeException("Wrong operator type found in post pass."); - } - } - - SingleInputOperator singleInputOperator = - (SingleInputOperator) sn.getOptimizerNode().getOperator(); - - // parameterize the node's driver strategy - for (int i = 0; i < sn.getDriverStrategy().getNumRequiredComparators(); i++) { - sn.setComparator( - createComparator( - singleInputOperator.getOperatorInfo().getInputType(), - sn.getKeys(i), - getSortOrders(sn.getKeys(i), sn.getSortOrders(i))), - i); - } - // done, we can now propagate our info down - traverseChannel(sn.getInput()); - - // don't forget the broadcast inputs - for (Channel c : sn.getBroadcastInputs()) { - traverseChannel(c); - } - } else if (node instanceof DualInputPlanNode) { - DualInputPlanNode dn = (DualInputPlanNode) node; - - if (!(dn.getOptimizerNode().getOperator() instanceof DualInputOperator)) { - throw new RuntimeException("Wrong operator type found in post pass."); - } - - DualInputOperator dualInputOperator = - (DualInputOperator) dn.getOptimizerNode().getOperator(); - - // parameterize the node's driver strategy - if (dn.getDriverStrategy().getNumRequiredComparators() > 0) { - dn.setComparator1( - createComparator( - dualInputOperator.getOperatorInfo().getFirstInputType(), - dn.getKeysForInput1(), - getSortOrders(dn.getKeysForInput1(), dn.getSortOrders()))); - dn.setComparator2( - createComparator( - dualInputOperator.getOperatorInfo().getSecondInputType(), - dn.getKeysForInput2(), - getSortOrders(dn.getKeysForInput2(), dn.getSortOrders()))); - - dn.setPairComparator( - createPairComparator( - dualInputOperator.getOperatorInfo().getFirstInputType(), - dualInputOperator.getOperatorInfo().getSecondInputType())); - } - - traverseChannel(dn.getInput1()); - traverseChannel(dn.getInput2()); - - // don't forget the broadcast inputs - for (Channel c : dn.getBroadcastInputs()) { - traverseChannel(c); - } - - } - // catch the sources of the iterative step functions - else if (node instanceof BulkPartialSolutionPlanNode - || node instanceof SolutionSetPlanNode - || node instanceof WorksetPlanNode) { - // Do nothing :D - } else if (node instanceof NAryUnionPlanNode) { - // Traverse to all child channels - for (Channel channel : node.getInputs()) { - traverseChannel(channel); - } - } else { - throw new CompilerPostPassException( - "Unknown node type encountered: " + node.getClass().getName()); - } - } - - private void traverseChannel(Channel channel) { - - PlanNode source = channel.getSource(); - Operator javaOp = source.getProgramOperator(); - - // if (!(javaOp instanceof BulkIteration) && !(javaOp instanceof JavaPlanNode)) { - // throw new RuntimeException("Wrong operator type found in post pass: " + javaOp); - // } - - TypeInformation type = javaOp.getOperatorInfo().getOutputType(); - - if (javaOp instanceof GroupReduceOperatorBase - && (source.getDriverStrategy() == DriverStrategy.SORTED_GROUP_COMBINE - || source.getDriverStrategy() == DriverStrategy.ALL_GROUP_REDUCE_COMBINE)) { - GroupReduceOperatorBase groupNode = (GroupReduceOperatorBase) javaOp; - type = groupNode.getInput().getOperatorInfo().getOutputType(); - } else if (javaOp instanceof PlanUnwrappingReduceGroupOperator - && source.getDriverStrategy().equals(DriverStrategy.SORTED_GROUP_COMBINE)) { - PlanUnwrappingReduceGroupOperator groupNode = - (PlanUnwrappingReduceGroupOperator) javaOp; - type = groupNode.getInput().getOperatorInfo().getOutputType(); - } - - // the serializer always exists - channel.setSerializer(createSerializer(type)); - - // parameterize the ship strategy - if (channel.getShipStrategy().requiresComparator()) { - channel.setShipStrategyComparator( - createComparator( - type, - channel.getShipStrategyKeys(), - getSortOrders( - channel.getShipStrategyKeys(), - channel.getShipStrategySortOrder()))); - } - - // parameterize the local strategy - if (channel.getLocalStrategy().requiresComparator()) { - channel.setLocalStrategyComparator( - createComparator( - type, - channel.getLocalStrategyKeys(), - getSortOrders( - channel.getLocalStrategyKeys(), - channel.getLocalStrategySortOrder()))); - } - - // descend to the channel's source - traverse(channel.getSource()); - } - - @SuppressWarnings("unchecked") - private static TypeInformation getTypeInfoFromSource(SourcePlanNode node) { - Operator op = node.getOptimizerNode().getOperator(); - - if (op instanceof GenericDataSourceBase) { - return ((GenericDataSourceBase) op).getOperatorInfo().getOutputType(); - } else { - throw new RuntimeException("Wrong operator type found in post pass."); - } - } - - private TypeSerializerFactory createSerializer(TypeInformation typeInfo) { - TypeSerializer serializer = - typeInfo.createSerializer(executionConfig.getSerializerConfig()); - - return new RuntimeSerializerFactory(serializer, typeInfo.getTypeClass()); - } - - @SuppressWarnings("unchecked") - private TypeComparatorFactory createComparator( - TypeInformation typeInfo, FieldList keys, boolean[] sortOrder) { - - TypeComparator comparator; - if (typeInfo instanceof CompositeType) { - comparator = - ((CompositeType) typeInfo) - .createComparator(keys.toArray(), sortOrder, 0, executionConfig); - } else if (typeInfo instanceof AtomicType) { - // handle grouping of atomic types - comparator = ((AtomicType) typeInfo).createComparator(sortOrder[0], executionConfig); - } else { - throw new RuntimeException("Unrecognized type: " + typeInfo); - } - - return new RuntimeComparatorFactory(comparator); - } - - private static - TypePairComparatorFactory createPairComparator( - TypeInformation typeInfo1, TypeInformation typeInfo2) { - // @SuppressWarnings("unchecked") - // TupleTypeInfo info1 = (TupleTypeInfo) typeInfo1; - // @SuppressWarnings("unchecked") - // TupleTypeInfo info2 = (TupleTypeInfo) typeInfo2; - - return new RuntimePairComparatorFactory(); - } - - private static final boolean[] getSortOrders(FieldList keys, boolean[] orders) { - if (orders == null) { - orders = new boolean[keys.size()]; - Arrays.fill(orders, true); - } - return orders; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/MissingFieldTypeInfoException.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/MissingFieldTypeInfoException.java deleted file mode 100644 index 85a05d37e8cfb..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/MissingFieldTypeInfoException.java +++ /dev/null @@ -1,34 +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.optimizer.postpass; - -public final class MissingFieldTypeInfoException extends Exception { - - private static final long serialVersionUID = 8749941961302509358L; - - private final int fieldNumber; - - public MissingFieldTypeInfoException(int fieldNumber) { - this.fieldNumber = fieldNumber; - } - - public int getFieldNumber() { - return fieldNumber; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/OptimizerPostPass.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/OptimizerPostPass.java deleted file mode 100644 index 6e60111ed42e0..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/OptimizerPostPass.java +++ /dev/null @@ -1,37 +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.optimizer.postpass; - -import org.apache.flink.optimizer.plan.OptimizedPlan; - -/** - * Interface for visitors that process the optimizer's plan. Typical post processing applications - * are schema finalization or the generation/parameterization of utilities for the actual data - * model. - */ -public interface OptimizerPostPass { - - /** - * Central post processing function. Invoked by the optimizer after the best plan has been - * determined. - * - * @param plan The plan to be post processed. - */ - void postPass(OptimizedPlan plan); -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/BinaryUnionReplacer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/BinaryUnionReplacer.java deleted file mode 100644 index e9ca4899d9df1..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/BinaryUnionReplacer.java +++ /dev/null @@ -1,131 +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.optimizer.traversals; - -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.plan.BinaryUnionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.IterationPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** - * A traversal that collects cascading binary unions into a single n-ary union operator. The - * exception is, when on of the union inputs is materialized, such as in the static-code-path-cache - * in iterations. - */ -public class BinaryUnionReplacer implements Visitor { - - private final Set seenBefore = new HashSet(); - - @Override - public boolean preVisit(PlanNode visitable) { - if (this.seenBefore.add(visitable)) { - if (visitable instanceof IterationPlanNode) { - ((IterationPlanNode) visitable).acceptForStepFunction(this); - } - return true; - } else { - return false; - } - } - - @Override - public void postVisit(PlanNode visitable) { - - if (visitable instanceof BinaryUnionPlanNode) { - - final BinaryUnionPlanNode unionNode = (BinaryUnionPlanNode) visitable; - final Channel in1 = unionNode.getInput1(); - final Channel in2 = unionNode.getInput2(); - - if (!unionNode.unionsStaticAndDynamicPath()) { - - // both on static path, or both on dynamic path. we can collapse them - NAryUnionPlanNode newUnionNode; - - List inputs = new ArrayList(); - collect(in1, inputs); - collect(in2, inputs); - - newUnionNode = - new NAryUnionPlanNode( - unionNode.getOptimizerNode(), - inputs, - unionNode.getGlobalProperties(), - unionNode.getCumulativeCosts()); - - newUnionNode.setParallelism(unionNode.getParallelism()); - - for (Channel c : inputs) { - c.setTarget(newUnionNode); - } - - for (Channel channel : unionNode.getOutgoingChannels()) { - channel.swapUnionNodes(newUnionNode); - newUnionNode.addOutgoingChannel(channel); - } - } else { - // union between the static and the dynamic path. we need to handle this for now - // through a special union operator - - // make sure that the first input is the cached (static) and the second input is the - // dynamic - if (in1.isOnDynamicPath()) { - BinaryUnionPlanNode newUnionNode = new BinaryUnionPlanNode(unionNode); - - in1.setTarget(newUnionNode); - in2.setTarget(newUnionNode); - - for (Channel channel : unionNode.getOutgoingChannels()) { - channel.swapUnionNodes(newUnionNode); - newUnionNode.addOutgoingChannel(channel); - } - } - } - } - } - - public void collect(Channel in, List inputs) { - if (in.getSource() instanceof NAryUnionPlanNode) { - // sanity check - if (in.getShipStrategy() != ShipStrategyType.FORWARD) { - throw new CompilerException( - "Bug: Plan generation for Unions picked a ship strategy between binary plan operators."); - } - if (!(in.getLocalStrategy() == null || in.getLocalStrategy() == LocalStrategy.NONE)) { - throw new CompilerException( - "Bug: Plan generation for Unions picked a local strategy between binary plan operators."); - } - - inputs.addAll(((NAryUnionPlanNode) in.getSource()).getListOfInputs()); - } else { - // is not a collapsed union node, so we take the channel directly - inputs.add(in); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/BranchesVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/BranchesVisitor.java deleted file mode 100644 index 0af11fb225146..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/BranchesVisitor.java +++ /dev/null @@ -1,45 +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.optimizer.traversals; - -import org.apache.flink.optimizer.dag.IterationNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.util.Visitor; - -/** - * This traversal of the optimizer DAG computes the information needed to track branches and joins - * in the data flow. This is important to support plans that are not a minimally connected DAG (Such - * plans are not trees, but at least one node feeds its output into more than one other node). - */ -public final class BranchesVisitor implements Visitor { - - @Override - public boolean preVisit(OptimizerNode node) { - return node.getOpenBranches() == null; - } - - @Override - public void postVisit(OptimizerNode node) { - if (node instanceof IterationNode) { - ((IterationNode) node).acceptForStepFunction(this); - } - - node.computeUnclosedBranchStack(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java deleted file mode 100644 index d209b12c15332..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/GraphCreatingVisitor.java +++ /dev/null @@ -1,428 +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.optimizer.traversals; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.operators.GenericDataSinkBase; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.Union; -import org.apache.flink.api.common.operators.base.BulkIterationBase; -import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; -import org.apache.flink.api.common.operators.base.CoGroupRawOperatorBase; -import org.apache.flink.api.common.operators.base.CrossOperatorBase; -import org.apache.flink.api.common.operators.base.DeltaIterationBase; -import org.apache.flink.api.common.operators.base.FilterOperatorBase; -import org.apache.flink.api.common.operators.base.FlatMapOperatorBase; -import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.base.MapPartitionOperatorBase; -import org.apache.flink.api.common.operators.base.OuterJoinOperatorBase; -import org.apache.flink.api.common.operators.base.PartitionOperatorBase; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.api.common.operators.base.SortPartitionOperatorBase; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.dag.BinaryUnionNode; -import org.apache.flink.optimizer.dag.BulkIterationNode; -import org.apache.flink.optimizer.dag.BulkPartialSolutionNode; -import org.apache.flink.optimizer.dag.CoGroupNode; -import org.apache.flink.optimizer.dag.CoGroupRawNode; -import org.apache.flink.optimizer.dag.CrossNode; -import org.apache.flink.optimizer.dag.DagConnection; -import org.apache.flink.optimizer.dag.DataSinkNode; -import org.apache.flink.optimizer.dag.DataSourceNode; -import org.apache.flink.optimizer.dag.FilterNode; -import org.apache.flink.optimizer.dag.FlatMapNode; -import org.apache.flink.optimizer.dag.GroupCombineNode; -import org.apache.flink.optimizer.dag.GroupReduceNode; -import org.apache.flink.optimizer.dag.JoinNode; -import org.apache.flink.optimizer.dag.MapNode; -import org.apache.flink.optimizer.dag.MapPartitionNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.OuterJoinNode; -import org.apache.flink.optimizer.dag.PartitionNode; -import org.apache.flink.optimizer.dag.ReduceNode; -import org.apache.flink.optimizer.dag.SolutionSetNode; -import org.apache.flink.optimizer.dag.SortPartitionNode; -import org.apache.flink.optimizer.dag.WorksetIterationNode; -import org.apache.flink.optimizer.dag.WorksetNode; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This traversal creates the optimizer DAG from a program. It works as a visitor that walks the - * program's flow in a depth-first fashion, starting from the data sinks. During the descent it - * creates an optimizer node for each operator, respectively data source or sink. During the ascent - * it connects the nodes to the full graph. - */ -public class GraphCreatingVisitor implements Visitor> { - - private final Map, OptimizerNode> - con2node; // map from the operator objects to their - // corresponding optimizer nodes - - private final List sinks; // all data sink nodes in the optimizer plan - - private final int defaultParallelism; // the default parallelism - - private final GraphCreatingVisitor - parent; // reference to enclosing creator, in case of a recursive translation - - private final ExecutionMode defaultDataExchangeMode; - - private final boolean forceParallelism; - - public GraphCreatingVisitor(int defaultParallelism, ExecutionMode defaultDataExchangeMode) { - this(null, false, defaultParallelism, defaultDataExchangeMode, null); - } - - private GraphCreatingVisitor( - GraphCreatingVisitor parent, - boolean forceParallelism, - int defaultParallelism, - ExecutionMode dataExchangeMode, - HashMap, OptimizerNode> closure) { - if (closure == null) { - con2node = new HashMap, OptimizerNode>(); - } else { - con2node = closure; - } - - this.sinks = new ArrayList(2); - this.defaultParallelism = defaultParallelism; - this.parent = parent; - this.defaultDataExchangeMode = dataExchangeMode; - this.forceParallelism = forceParallelism; - } - - public List getSinks() { - return sinks; - } - - @SuppressWarnings("deprecation") - @Override - public boolean preVisit(Operator c) { - // check if we have been here before - if (this.con2node.containsKey(c)) { - return false; - } - - final OptimizerNode n; - - // create a node for the operator (or sink or source) if we have not been here before - if (c instanceof GenericDataSinkBase) { - DataSinkNode dsn = new DataSinkNode((GenericDataSinkBase) c); - this.sinks.add(dsn); - n = dsn; - } else if (c instanceof GenericDataSourceBase) { - n = new DataSourceNode((GenericDataSourceBase) c); - } else if (c instanceof MapOperatorBase) { - n = new MapNode((MapOperatorBase) c); - } else if (c instanceof MapPartitionOperatorBase) { - n = new MapPartitionNode((MapPartitionOperatorBase) c); - } else if (c instanceof FlatMapOperatorBase) { - n = new FlatMapNode((FlatMapOperatorBase) c); - } else if (c instanceof FilterOperatorBase) { - n = new FilterNode((FilterOperatorBase) c); - } else if (c instanceof ReduceOperatorBase) { - n = new ReduceNode((ReduceOperatorBase) c); - } else if (c instanceof GroupCombineOperatorBase) { - n = new GroupCombineNode((GroupCombineOperatorBase) c); - } else if (c instanceof GroupReduceOperatorBase) { - n = new GroupReduceNode((GroupReduceOperatorBase) c); - } else if (c instanceof InnerJoinOperatorBase) { - n = new JoinNode((InnerJoinOperatorBase) c); - } else if (c instanceof OuterJoinOperatorBase) { - n = new OuterJoinNode((OuterJoinOperatorBase) c); - } else if (c instanceof CoGroupOperatorBase) { - n = new CoGroupNode((CoGroupOperatorBase) c); - } else if (c instanceof CoGroupRawOperatorBase) { - n = new CoGroupRawNode((CoGroupRawOperatorBase) c); - } else if (c instanceof CrossOperatorBase) { - n = new CrossNode((CrossOperatorBase) c); - } else if (c instanceof BulkIterationBase) { - n = new BulkIterationNode((BulkIterationBase) c); - } else if (c instanceof DeltaIterationBase) { - n = new WorksetIterationNode((DeltaIterationBase) c); - } else if (c instanceof Union) { - n = new BinaryUnionNode((Union) c); - } else if (c instanceof PartitionOperatorBase) { - n = new PartitionNode((PartitionOperatorBase) c); - } else if (c instanceof SortPartitionOperatorBase) { - n = new SortPartitionNode((SortPartitionOperatorBase) c); - } else if (c instanceof BulkIterationBase.PartialSolutionPlaceHolder) { - if (this.parent == null) { - throw new InvalidProgramException( - "It is currently not supported to create data sinks inside iterations."); - } - - final BulkIterationBase.PartialSolutionPlaceHolder holder = - (BulkIterationBase.PartialSolutionPlaceHolder) c; - final BulkIterationBase enclosingIteration = holder.getContainingBulkIteration(); - final BulkIterationNode containingIterationNode = - (BulkIterationNode) this.parent.con2node.get(enclosingIteration); - - // catch this for the recursive translation of step functions - BulkPartialSolutionNode p = - new BulkPartialSolutionNode(holder, containingIterationNode); - p.setParallelism(containingIterationNode.getParallelism()); - n = p; - } else if (c instanceof DeltaIterationBase.WorksetPlaceHolder) { - if (this.parent == null) { - throw new InvalidProgramException( - "It is currently not supported to create data sinks inside iterations."); - } - - final DeltaIterationBase.WorksetPlaceHolder holder = - (DeltaIterationBase.WorksetPlaceHolder) c; - final DeltaIterationBase enclosingIteration = - holder.getContainingWorksetIteration(); - final WorksetIterationNode containingIterationNode = - (WorksetIterationNode) this.parent.con2node.get(enclosingIteration); - - // catch this for the recursive translation of step functions - WorksetNode p = new WorksetNode(holder, containingIterationNode); - p.setParallelism(containingIterationNode.getParallelism()); - n = p; - } else if (c instanceof DeltaIterationBase.SolutionSetPlaceHolder) { - if (this.parent == null) { - throw new InvalidProgramException( - "It is currently not supported to create data sinks inside iterations."); - } - - final DeltaIterationBase.SolutionSetPlaceHolder holder = - (DeltaIterationBase.SolutionSetPlaceHolder) c; - final DeltaIterationBase enclosingIteration = - holder.getContainingWorksetIteration(); - final WorksetIterationNode containingIterationNode = - (WorksetIterationNode) this.parent.con2node.get(enclosingIteration); - - // catch this for the recursive translation of step functions - SolutionSetNode p = new SolutionSetNode(holder, containingIterationNode); - p.setParallelism(containingIterationNode.getParallelism()); - n = p; - } else { - throw new IllegalArgumentException("Unknown operator type: " + c); - } - - this.con2node.put(c, n); - - // set the parallelism only if it has not been set before. some nodes have a fixed - // parallelism, such as the - // key-less reducer (all-reduce) - if (n.getParallelism() < 1) { - // set the parallelism - int par = c.getParallelism(); - if (n instanceof BinaryUnionNode) { - // Keep parallelism of union undefined for now. - // It will be determined based on the parallelism of its successor. - par = -1; - } else if (par > 0) { - if (this.forceParallelism && par != this.defaultParallelism) { - par = this.defaultParallelism; - Optimizer.LOG.warn( - "The parallelism of nested dataflows (such as step functions in iterations) is " - + "currently fixed to the parallelism of the surrounding operator (the iteration)."); - } - } else { - par = this.defaultParallelism; - } - n.setParallelism(par); - } - - return true; - } - - @Override - public void postVisit(Operator c) { - - OptimizerNode n = this.con2node.get(c); - - // first connect to the predecessors - n.setInput(this.con2node, this.defaultDataExchangeMode); - n.setBroadcastInputs(this.con2node, this.defaultDataExchangeMode); - - // if the node represents a bulk iteration, we recursively translate the data flow now - if (n instanceof BulkIterationNode) { - final BulkIterationNode iterNode = (BulkIterationNode) n; - final BulkIterationBase iter = iterNode.getIterationContract(); - - // pass a copy of the no iterative part into the iteration translation, - // in case the iteration references its closure - HashMap, OptimizerNode> closure = - new HashMap, OptimizerNode>(con2node); - - // first, recursively build the data flow for the step function - final GraphCreatingVisitor recursiveCreator = - new GraphCreatingVisitor( - this, - true, - iterNode.getParallelism(), - defaultDataExchangeMode, - closure); - - BulkPartialSolutionNode partialSolution; - - iter.getNextPartialSolution().accept(recursiveCreator); - - partialSolution = - (BulkPartialSolutionNode) - recursiveCreator.con2node.get(iter.getPartialSolution()); - OptimizerNode rootOfStepFunction = - recursiveCreator.con2node.get(iter.getNextPartialSolution()); - if (partialSolution == null) { - throw new CompilerException( - "Error: The step functions result does not depend on the partial solution."); - } - - OptimizerNode terminationCriterion = null; - - if (iter.getTerminationCriterion() != null) { - terminationCriterion = - recursiveCreator.con2node.get(iter.getTerminationCriterion()); - - // no intermediate node yet, traverse from the termination criterion to build the - // missing parts - if (terminationCriterion == null) { - iter.getTerminationCriterion().accept(recursiveCreator); - terminationCriterion = - recursiveCreator.con2node.get(iter.getTerminationCriterion()); - } - } - - iterNode.setPartialSolution(partialSolution); - iterNode.setNextPartialSolution(rootOfStepFunction, terminationCriterion); - - // go over the contained data flow and mark the dynamic path nodes - StaticDynamicPathIdentifier identifier = - new StaticDynamicPathIdentifier(iterNode.getCostWeight()); - iterNode.acceptForStepFunction(identifier); - } else if (n instanceof WorksetIterationNode) { - final WorksetIterationNode iterNode = (WorksetIterationNode) n; - final DeltaIterationBase iter = iterNode.getIterationContract(); - - // we need to ensure that both the next-workset and the solution-set-delta depend on the - // workset. - // One check is for free during the translation, we do the other check here as a - // pre-condition - { - StepFunctionValidator wsf = new StepFunctionValidator(); - iter.getNextWorkset().accept(wsf); - if (!wsf.hasFoundWorkset()) { - throw new CompilerException( - "In the given program, the next workset does not depend on the workset. " - + "This is a prerequisite in delta iterations."); - } - } - - // calculate the closure of the anonymous function - HashMap, OptimizerNode> closure = - new HashMap, OptimizerNode>(con2node); - - // first, recursively build the data flow for the step function - final GraphCreatingVisitor recursiveCreator = - new GraphCreatingVisitor( - this, - true, - iterNode.getParallelism(), - defaultDataExchangeMode, - closure); - - // descend from the solution set delta. check that it depends on both the workset - // and the solution set. If it does depend on both, this descend should create both - // nodes - iter.getSolutionSetDelta().accept(recursiveCreator); - - final WorksetNode worksetNode = - (WorksetNode) recursiveCreator.con2node.get(iter.getWorkset()); - - if (worksetNode == null) { - throw new CompilerException( - "In the given program, the solution set delta does not depend on the workset." - + "This is a prerequisite in delta iterations."); - } - - iter.getNextWorkset().accept(recursiveCreator); - - SolutionSetNode solutionSetNode = - (SolutionSetNode) recursiveCreator.con2node.get(iter.getSolutionSet()); - - if (solutionSetNode == null - || solutionSetNode.getOutgoingConnections() == null - || solutionSetNode.getOutgoingConnections().isEmpty()) { - solutionSetNode = - new SolutionSetNode( - (DeltaIterationBase.SolutionSetPlaceHolder) - iter.getSolutionSet(), - iterNode); - } else { - for (DagConnection conn : solutionSetNode.getOutgoingConnections()) { - OptimizerNode successor = conn.getTarget(); - - if (successor.getClass() == JoinNode.class) { - // find out which input to the match the solution set is - JoinNode mn = (JoinNode) successor; - if (mn.getFirstPredecessorNode() == solutionSetNode) { - mn.makeJoinWithSolutionSet(0); - } else if (mn.getSecondPredecessorNode() == solutionSetNode) { - mn.makeJoinWithSolutionSet(1); - } else { - throw new CompilerException(); - } - } else if (successor.getClass() == CoGroupNode.class) { - CoGroupNode cg = (CoGroupNode) successor; - if (cg.getFirstPredecessorNode() == solutionSetNode) { - cg.makeCoGroupWithSolutionSet(0); - } else if (cg.getSecondPredecessorNode() == solutionSetNode) { - cg.makeCoGroupWithSolutionSet(1); - } else { - throw new CompilerException(); - } - } else { - throw new InvalidProgramException( - "Error: The only operations allowed on the solution set are Join and CoGroup."); - } - } - } - - final OptimizerNode nextWorksetNode = - recursiveCreator.con2node.get(iter.getNextWorkset()); - final OptimizerNode solutionSetDeltaNode = - recursiveCreator.con2node.get(iter.getSolutionSetDelta()); - - // set the step function nodes to the iteration node - iterNode.setPartialSolution(solutionSetNode, worksetNode); - iterNode.setNextPartialSolution( - solutionSetDeltaNode, nextWorksetNode, defaultDataExchangeMode); - - // go over the contained data flow and mark the dynamic path nodes - StaticDynamicPathIdentifier pathIdentifier = - new StaticDynamicPathIdentifier(iterNode.getCostWeight()); - iterNode.acceptForStepFunction(pathIdentifier); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/IdAndEstimatesVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/IdAndEstimatesVisitor.java deleted file mode 100644 index f6418aa142339..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/IdAndEstimatesVisitor.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.optimizer.traversals; - -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.dag.DagConnection; -import org.apache.flink.optimizer.dag.IterationNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.util.Visitor; - -/** - * This traversal of the optimizer DAG assigns IDs to each node (in a pre-order fashion), and calls - * each node to compute its estimates. The latter happens in the postVisit function, where it is - * guaranteed that all predecessors have computed their estimates. - */ -public class IdAndEstimatesVisitor implements Visitor { - - private final DataStatistics statistics; - - private int id = 1; - - public IdAndEstimatesVisitor(DataStatistics statistics) { - this.statistics = statistics; - } - - @Override - public boolean preVisit(OptimizerNode visitable) { - return visitable.getId() == -1; - } - - @Override - public void postVisit(OptimizerNode visitable) { - // the node ids - visitable.initId(this.id++); - - // connections need to figure out their maximum path depths - for (DagConnection conn : visitable.getIncomingConnections()) { - conn.initMaxDepth(); - } - for (DagConnection conn : visitable.getBroadcastConnections()) { - conn.initMaxDepth(); - } - - // the estimates - visitable.computeOutputEstimates(this.statistics); - - // if required, recurse into the step function - if (visitable instanceof IterationNode) { - ((IterationNode) visitable).acceptForStepFunction(this); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/InterestingPropertyVisitor.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/InterestingPropertyVisitor.java deleted file mode 100644 index 34ee32c660ac2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/InterestingPropertyVisitor.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.optimizer.traversals; - -import org.apache.flink.optimizer.costs.CostEstimator; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.util.Visitor; - -/** - * Visitor that computes the interesting properties for each node in the optimizer DAG. On its - * recursive depth-first descend, it propagates all interesting properties top-down. - */ -public class InterestingPropertyVisitor implements Visitor { - - private CostEstimator - estimator; // the cost estimator for maximal costs of an interesting property - - /** - * Creates a new visitor that computes the interesting properties for all nodes in the plan. It - * uses the given cost estimator used to compute the maximal costs for an interesting property. - * - * @param estimator The cost estimator to estimate the maximal costs for interesting properties. - */ - public InterestingPropertyVisitor(CostEstimator estimator) { - this.estimator = estimator; - } - - @Override - public boolean preVisit(OptimizerNode node) { - // The interesting properties must be computed on the descend. In case a node has multiple - // outputs, - // that computation must happen during the last descend. - - if (node.getInterestingProperties() == null - && node.haveAllOutputConnectionInterestingProperties()) { - node.computeUnionOfInterestingPropertiesFromSuccessors(); - node.computeInterestingPropertiesForInputs(this.estimator); - return true; - } else { - return false; - } - } - - @Override - public void postVisit(OptimizerNode visitable) {} -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/PlanFinalizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/PlanFinalizer.java deleted file mode 100644 index 10542d360466a..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/PlanFinalizer.java +++ /dev/null @@ -1,240 +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.optimizer.traversals; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.BinaryUnionPlanNode; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.IterationPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SolutionSetPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plan.WorksetPlanNode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Visitor; - -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Deque; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** - * This visitor traverses the selected execution plan and finalizes it: - * - *

    - *
  • The graph of nodes is double-linked (links from child to parent are inserted). - *
  • If unions join static and dynamic paths, the cache is marked as a memory consumer. - *
  • Relative memory fractions are assigned to all nodes. - *
  • All nodes are collected into a set. - *
- */ -public class PlanFinalizer implements Visitor { - - private final Set allNodes; // a set of all nodes in the optimizer plan - - private final List sources; // all data source nodes in the optimizer plan - - private final List sinks; // all data sink nodes in the optimizer plan - - private final Deque stackOfIterationNodes; - - private int memoryConsumerWeights; // a counter of all memory consumers - - /** Creates a new plan finalizer. */ - public PlanFinalizer() { - this.allNodes = new HashSet(); - this.sources = new ArrayList(); - this.sinks = new ArrayList(); - this.stackOfIterationNodes = new ArrayDeque(); - } - - public OptimizedPlan createFinalPlan( - List sinks, String jobName, Plan originalPlan) { - this.memoryConsumerWeights = 0; - - // traverse the graph - for (SinkPlanNode node : sinks) { - node.accept(this); - } - - // assign the memory to each node - if (this.memoryConsumerWeights > 0) { - for (PlanNode node : this.allNodes) { - // assign memory to the driver strategy of the node - final int consumerWeight = node.getMemoryConsumerWeight(); - if (consumerWeight > 0) { - final double relativeMem = (double) consumerWeight / this.memoryConsumerWeights; - node.setRelativeMemoryPerSubtask(relativeMem); - if (Optimizer.LOG.isDebugEnabled()) { - Optimizer.LOG.debug( - "Assigned " - + relativeMem - + " of total memory to each subtask of " - + node.getProgramOperator().getName() - + "."); - } - } - - // assign memory to the local and global strategies of the channels - for (Channel c : node.getInputs()) { - if (c.getLocalStrategy().dams()) { - final double relativeMem = 1.0 / this.memoryConsumerWeights; - c.setRelativeMemoryLocalStrategy(relativeMem); - if (Optimizer.LOG.isDebugEnabled()) { - Optimizer.LOG.debug( - "Assigned " - + relativeMem - + " of total memory to each local strategy " - + "instance of " - + c - + "."); - } - } - if (c.getTempMode() != TempMode.NONE) { - final double relativeMem = 1.0 / this.memoryConsumerWeights; - c.setRelativeTempMemory(relativeMem); - if (Optimizer.LOG.isDebugEnabled()) { - Optimizer.LOG.debug( - "Assigned " - + relativeMem - + " of total memory to each instance of the temp " - + "table for " - + c - + "."); - } - } - } - } - } - return new OptimizedPlan(this.sources, this.sinks, this.allNodes, jobName, originalPlan); - } - - @Override - public boolean preVisit(PlanNode visitable) { - // if we come here again, prevent a further descend - if (!this.allNodes.add(visitable)) { - return false; - } - - if (visitable instanceof SinkPlanNode) { - this.sinks.add((SinkPlanNode) visitable); - } else if (visitable instanceof SourcePlanNode) { - this.sources.add((SourcePlanNode) visitable); - } else if (visitable instanceof BinaryUnionPlanNode) { - BinaryUnionPlanNode unionNode = (BinaryUnionPlanNode) visitable; - if (unionNode.unionsStaticAndDynamicPath()) { - unionNode.setDriverStrategy(DriverStrategy.UNION_WITH_CACHED); - } - } else if (visitable instanceof BulkPartialSolutionPlanNode) { - // tell the partial solution about the iteration node that contains it - final BulkPartialSolutionPlanNode pspn = (BulkPartialSolutionPlanNode) visitable; - final IterationPlanNode iteration = this.stackOfIterationNodes.peekLast(); - - // sanity check! - if (!(iteration instanceof BulkIterationPlanNode)) { - throw new CompilerException( - "Bug: Error finalizing the plan. " - + "Cannot associate the node for a partial solutions with its containing iteration."); - } - pspn.setContainingIterationNode((BulkIterationPlanNode) iteration); - } else if (visitable instanceof WorksetPlanNode) { - // tell the partial solution about the iteration node that contains it - final WorksetPlanNode wspn = (WorksetPlanNode) visitable; - final IterationPlanNode iteration = this.stackOfIterationNodes.peekLast(); - - // sanity check! - if (!(iteration instanceof WorksetIterationPlanNode)) { - throw new CompilerException( - "Bug: Error finalizing the plan. " - + "Cannot associate the node for a partial solutions with its containing iteration."); - } - wspn.setContainingIterationNode((WorksetIterationPlanNode) iteration); - } else if (visitable instanceof SolutionSetPlanNode) { - // tell the partial solution about the iteration node that contains it - final SolutionSetPlanNode sspn = (SolutionSetPlanNode) visitable; - final IterationPlanNode iteration = this.stackOfIterationNodes.peekLast(); - - // sanity check! - if (!(iteration instanceof WorksetIterationPlanNode)) { - throw new CompilerException( - "Bug: Error finalizing the plan. " - + "Cannot associate the node for a partial solutions with its containing iteration."); - } - sspn.setContainingIterationNode((WorksetIterationPlanNode) iteration); - } - - // double-connect the connections. previously, only parents knew their children, because - // one child candidate could have been referenced by multiple parents. - for (Channel conn : visitable.getInputs()) { - conn.setTarget(visitable); - conn.getSource().addOutgoingChannel(conn); - } - - for (Channel c : visitable.getBroadcastInputs()) { - c.setTarget(visitable); - c.getSource().addOutgoingChannel(c); - } - - // count the memory consumption - this.memoryConsumerWeights += visitable.getMemoryConsumerWeight(); - for (Channel c : visitable.getInputs()) { - if (c.getLocalStrategy().dams()) { - this.memoryConsumerWeights++; - } - if (c.getTempMode() != TempMode.NONE) { - this.memoryConsumerWeights++; - } - } - for (Channel c : visitable.getBroadcastInputs()) { - if (c.getLocalStrategy().dams()) { - this.memoryConsumerWeights++; - } - if (c.getTempMode() != TempMode.NONE) { - this.memoryConsumerWeights++; - } - } - - // pass the visitor to the iteration's step function - if (visitable instanceof IterationPlanNode) { - // push the iteration node onto the stack - final IterationPlanNode iterNode = (IterationPlanNode) visitable; - this.stackOfIterationNodes.addLast(iterNode); - - // recurse - ((IterationPlanNode) visitable).acceptForStepFunction(this); - - // pop the iteration node from the stack - this.stackOfIterationNodes.removeLast(); - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) {} -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java deleted file mode 100644 index 3dc144c5474d6..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java +++ /dev/null @@ -1,266 +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.optimizer.traversals; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.distributions.CommonRangeBoundaries; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.UnaryOperatorInformation; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.base.MapPartitionOperatorBase; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.java.functions.IdPartitioner; -import org.apache.flink.api.java.functions.SampleInCoordinator; -import org.apache.flink.api.java.functions.SampleInPartition; -import org.apache.flink.api.java.sampling.IntermediateSampleData; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.optimizer.costs.Costs; -import org.apache.flink.optimizer.dag.GroupReduceNode; -import org.apache.flink.optimizer.dag.MapNode; -import org.apache.flink.optimizer.dag.MapPartitionNode; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.IterationPlanNode; -import org.apache.flink.optimizer.plan.NamedChannel; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.util.Utils; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.udf.AssignRangeIndex; -import org.apache.flink.runtime.operators.udf.RangeBoundaryBuilder; -import org.apache.flink.runtime.operators.udf.RemoveRangeIndex; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** */ -public class RangePartitionRewriter implements Visitor { - - static final long SEED = 0; - static final String SIP_NAME = "RangePartition: LocalSample"; - static final String SIC_NAME = "RangePartition: GlobalSample"; - static final String RB_NAME = "RangePartition: Histogram"; - static final String ARI_NAME = "RangePartition: PreparePartition"; - static final String PR_NAME = "RangePartition: Partition"; - - static final int SAMPLES_PER_PARTITION = 1000; - - static final IdPartitioner idPartitioner = new IdPartitioner(); - - final OptimizedPlan plan; - final Set visitedIterationNodes; - - public RangePartitionRewriter(OptimizedPlan plan) { - this.plan = plan; - this.visitedIterationNodes = new HashSet<>(); - } - - @Override - public boolean preVisit(PlanNode visitable) { - return true; - } - - @Override - public void postVisit(PlanNode node) { - - if (node instanceof IterationPlanNode) { - IterationPlanNode iNode = (IterationPlanNode) node; - if (!visitedIterationNodes.contains(iNode)) { - visitedIterationNodes.add(iNode); - iNode.acceptForStepFunction(this); - } - } - - final Iterable inputChannels = node.getInputs(); - for (Channel channel : inputChannels) { - ShipStrategyType shipStrategy = channel.getShipStrategy(); - // Make sure we only optimize the DAG for range partition, and do not optimize multi - // times. - if (shipStrategy == ShipStrategyType.PARTITION_RANGE) { - - if (channel.getDataDistribution() == null) { - if (node.isOnDynamicPath()) { - throw new InvalidProgramException( - "Range Partitioning not supported within iterations if users do not supply the data distribution."); - } - - PlanNode channelSource = channel.getSource(); - List newSourceOutputChannels = rewriteRangePartitionChannel(channel); - channelSource.getOutgoingChannels().remove(channel); - channelSource.getOutgoingChannels().addAll(newSourceOutputChannels); - } - } - } - } - - private List rewriteRangePartitionChannel(Channel channel) { - final List sourceNewOutputChannels = new ArrayList<>(); - final PlanNode sourceNode = channel.getSource(); - final PlanNode targetNode = channel.getTarget(); - final int sourceParallelism = sourceNode.getParallelism(); - final int targetParallelism = targetNode.getParallelism(); - final Costs defaultZeroCosts = new Costs(0, 0, 0); - final TypeComparatorFactory comparator = - Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig()); - // 1. Fixed size sample in each partitions. - final int sampleSize = SAMPLES_PER_PARTITION * targetParallelism; - final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, SEED); - final TypeInformation sourceOutputType = - sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType(); - final TypeInformation isdTypeInformation = - TypeExtractor.getForClass(IntermediateSampleData.class); - final UnaryOperatorInformation sipOperatorInformation = - new UnaryOperatorInformation(sourceOutputType, isdTypeInformation); - final MapPartitionOperatorBase sipOperatorBase = - new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, SIP_NAME); - final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase); - final Channel sipChannel = new Channel(sourceNode, TempMode.NONE); - sipChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - final SingleInputPlanNode sipPlanNode = - new SingleInputPlanNode( - sipNode, SIP_NAME, sipChannel, DriverStrategy.MAP_PARTITION); - sipNode.setParallelism(sourceParallelism); - sipPlanNode.setParallelism(sourceParallelism); - sipPlanNode.initProperties(new GlobalProperties(), new LocalProperties()); - sipPlanNode.setCosts(defaultZeroCosts); - sipChannel.setTarget(sipPlanNode); - this.plan.getAllNodes().add(sipPlanNode); - sourceNewOutputChannels.add(sipChannel); - - // 2. Fixed size sample in a single coordinator. - final SampleInCoordinator sampleInCoordinator = - new SampleInCoordinator(false, sampleSize, SEED); - final UnaryOperatorInformation sicOperatorInformation = - new UnaryOperatorInformation(isdTypeInformation, sourceOutputType); - final GroupReduceOperatorBase sicOperatorBase = - new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, SIC_NAME); - final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase); - final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE); - sicChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - final SingleInputPlanNode sicPlanNode = - new SingleInputPlanNode( - sicNode, SIC_NAME, sicChannel, DriverStrategy.ALL_GROUP_REDUCE); - sicNode.setParallelism(1); - sicPlanNode.setParallelism(1); - sicPlanNode.initProperties(new GlobalProperties(), new LocalProperties()); - sicPlanNode.setCosts(defaultZeroCosts); - sicChannel.setTarget(sicPlanNode); - sipPlanNode.addOutgoingChannel(sicChannel); - this.plan.getAllNodes().add(sicPlanNode); - - // 3. Use sampled data to build range boundaries. - final RangeBoundaryBuilder rangeBoundaryBuilder = - new RangeBoundaryBuilder(comparator, targetParallelism); - final TypeInformation rbTypeInformation = - TypeExtractor.getForClass(CommonRangeBoundaries.class); - final UnaryOperatorInformation rbOperatorInformation = - new UnaryOperatorInformation(sourceOutputType, rbTypeInformation); - final MapPartitionOperatorBase rbOperatorBase = - new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, RB_NAME); - final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase); - final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE); - rbChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - final SingleInputPlanNode rbPlanNode = - new SingleInputPlanNode(rbNode, RB_NAME, rbChannel, DriverStrategy.MAP_PARTITION); - rbNode.setParallelism(1); - rbPlanNode.setParallelism(1); - rbPlanNode.initProperties(new GlobalProperties(), new LocalProperties()); - rbPlanNode.setCosts(defaultZeroCosts); - rbChannel.setTarget(rbPlanNode); - sicPlanNode.addOutgoingChannel(rbChannel); - this.plan.getAllNodes().add(rbPlanNode); - - // 4. Take range boundaries as broadcast input and take the tuple of partition id and record - // as output. - final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator); - final TypeInformation ariOutputTypeInformation = - new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType); - final UnaryOperatorInformation ariOperatorInformation = - new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation); - final MapPartitionOperatorBase ariOperatorBase = - new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, ARI_NAME); - final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase); - final Channel ariChannel = new Channel(sourceNode, TempMode.NONE); - // To avoid deadlock, set the DataExchangeMode of channel between source node and this to - // Batch. - ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH); - final SingleInputPlanNode ariPlanNode = - new SingleInputPlanNode( - ariNode, ARI_NAME, ariChannel, DriverStrategy.MAP_PARTITION); - ariNode.setParallelism(sourceParallelism); - ariPlanNode.setParallelism(sourceParallelism); - ariPlanNode.initProperties(new GlobalProperties(), new LocalProperties()); - ariPlanNode.setCosts(defaultZeroCosts); - ariChannel.setTarget(ariPlanNode); - this.plan.getAllNodes().add(ariPlanNode); - sourceNewOutputChannels.add(ariChannel); - - final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode); - broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, DataExchangeMode.PIPELINED); - broadcastChannel.setTarget(ariPlanNode); - List broadcastChannels = new ArrayList<>(1); - broadcastChannels.add(broadcastChannel); - ariPlanNode.setBroadcastInputs(broadcastChannels); - - // 5. Remove the partition id. - final Channel partChannel = new Channel(ariPlanNode, TempMode.NONE); - final FieldList keys = new FieldList(0); - partChannel.setShipStrategy( - ShipStrategyType.PARTITION_CUSTOM, keys, idPartitioner, DataExchangeMode.PIPELINED); - ariPlanNode.addOutgoingChannel(partChannel); - - final RemoveRangeIndex partitionIDRemoveWrapper = new RemoveRangeIndex(); - final UnaryOperatorInformation prOperatorInformation = - new UnaryOperatorInformation(ariOutputTypeInformation, sourceOutputType); - final MapOperatorBase prOperatorBase = - new MapOperatorBase(partitionIDRemoveWrapper, prOperatorInformation, PR_NAME); - final MapNode prRemoverNode = new MapNode(prOperatorBase); - final SingleInputPlanNode prPlanNode = - new SingleInputPlanNode(prRemoverNode, PR_NAME, partChannel, DriverStrategy.MAP); - partChannel.setTarget(prPlanNode); - prRemoverNode.setParallelism(targetParallelism); - prPlanNode.setParallelism(targetParallelism); - GlobalProperties globalProperties = new GlobalProperties(); - globalProperties.setRangePartitioned(new Ordering(0, null, Order.ASCENDING)); - prPlanNode.initProperties(globalProperties, new LocalProperties()); - prPlanNode.setCosts(defaultZeroCosts); - this.plan.getAllNodes().add(prPlanNode); - - // 6. Connect to target node. - channel.setSource(prPlanNode); - channel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - prPlanNode.addOutgoingChannel(channel); - - return sourceNewOutputChannels; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/StaticDynamicPathIdentifier.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/StaticDynamicPathIdentifier.java deleted file mode 100644 index f7b93efb57607..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/StaticDynamicPathIdentifier.java +++ /dev/null @@ -1,58 +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.optimizer.traversals; - -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.dag.IterationNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.util.Visitor; - -import java.util.HashSet; -import java.util.Set; - -/** - * A traversal that goes over the program data flow of an iteration and makes the nodes that depend - * on the partial solution (the data set recomputed in each iteration) as "dynamic" and the other - * nodes as "static". - */ -public class StaticDynamicPathIdentifier implements Visitor { - - private final Set seenBefore = new HashSet(); - - private final int costWeight; - - public StaticDynamicPathIdentifier(int costWeight) { - this.costWeight = costWeight; - } - - @Override - public boolean preVisit(OptimizerNode visitable) { - return this.seenBefore.add(visitable); - } - - @Override - public void postVisit(OptimizerNode visitable) { - visitable.identifyDynamicPath(this.costWeight); - - // check that there is no nested iteration on the dynamic path - if (visitable.isOnDynamicPath() && visitable instanceof IterationNode) { - throw new CompilerException("Nested iterations are currently not supported."); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/StepFunctionValidator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/StepFunctionValidator.java deleted file mode 100644 index d044e3f297640..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/StepFunctionValidator.java +++ /dev/null @@ -1,53 +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.optimizer.traversals; - -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.DeltaIterationBase; -import org.apache.flink.util.Visitor; - -import java.util.HashSet; -import java.util.Set; - -/** - * A traversal that checks if the Workset of a delta iteration is used in the data flow of its step - * function. - */ -public class StepFunctionValidator implements Visitor> { - - private final Set> seenBefore = new HashSet>(); - - private boolean foundWorkset; - - @Override - public boolean preVisit(Operator visitable) { - if (visitable instanceof DeltaIterationBase.WorksetPlaceHolder) { - foundWorkset = true; - } - - return (!foundWorkset) && seenBefore.add(visitable); - } - - @Override - public void postVisit(Operator visitable) {} - - public boolean hasFoundWorkset() { - return foundWorkset; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/UnionParallelismAndForwardEnforcer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/UnionParallelismAndForwardEnforcer.java deleted file mode 100644 index 4bc79fcab92d9..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/UnionParallelismAndForwardEnforcer.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.optimizer.traversals; - -import org.apache.flink.optimizer.dag.BinaryUnionNode; -import org.apache.flink.optimizer.dag.DagConnection; -import org.apache.flink.optimizer.dag.IterationNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -/** - * Enforces that all union nodes have the same parallelism as their successor (there must be only - * one!) and that the union node and its successor are connected by a forward ship strategy. - */ -public class UnionParallelismAndForwardEnforcer implements Visitor { - - @Override - public boolean preVisit(OptimizerNode node) { - - // if the current node is a union - if (node instanceof BinaryUnionNode) { - int parallelism = -1; - // set ship strategy of all outgoing connections to FORWARD. - for (DagConnection conn : node.getOutgoingConnections()) { - parallelism = conn.getTarget().getParallelism(); - conn.setShipStrategy(ShipStrategyType.FORWARD); - } - // adjust parallelism to be same as successor - node.setParallelism(parallelism); - } - - // traverse the whole plan - return true; - } - - @Override - public void postVisit(OptimizerNode node) { - // if required, recurse into the step function - if (node instanceof IterationNode) { - ((IterationNode) node).acceptForStepFunction(this); - } - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/package-info.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/package-info.java deleted file mode 100644 index f02dddf7f2399..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/package-info.java +++ /dev/null @@ -1,26 +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. - */ - -/** - * This package contains the various traversals over the program plan and the optimizer DAG - * (directed acyclic graph) that are made in the course of the optimization. - * - *

The traversals are mostly implemented as a {@link org.apache.flink.util.Visitor} that - * traversed the program flow. - */ -package org.apache.flink.optimizer.traversals; diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/NoOpBinaryUdfOp.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/NoOpBinaryUdfOp.java deleted file mode 100644 index 249ee30c523e2..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/NoOpBinaryUdfOp.java +++ /dev/null @@ -1,48 +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.optimizer.util; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.functions.util.NoOpFunction; -import org.apache.flink.api.common.operators.BinaryOperatorInformation; -import org.apache.flink.api.common.operators.DualInputOperator; -import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; -import org.apache.flink.api.common.typeinfo.TypeInformation; - -import java.util.List; - -public class NoOpBinaryUdfOp extends DualInputOperator { - - public NoOpBinaryUdfOp(TypeInformation type) { - super( - new UserCodeClassWrapper(NoOpFunction.class), - new BinaryOperatorInformation(type, type, type), - "NoContract"); - } - - @Override - protected List executeOnCollections( - List inputData1, - List inputData2, - RuntimeContext runtimeContext, - ExecutionConfig executionConfig) { - throw new UnsupportedOperationException(); - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/NoOpUnaryUdfOp.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/NoOpUnaryUdfOp.java deleted file mode 100644 index 7d63692f145e5..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/NoOpUnaryUdfOp.java +++ /dev/null @@ -1,53 +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.optimizer.util; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.functions.util.NoOpFunction; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.UnaryOperatorInformation; -import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; -import org.apache.flink.api.common.typeinfo.TypeInformation; - -import java.util.List; - -public class NoOpUnaryUdfOp extends SingleInputOperator { - - @SuppressWarnings("rawtypes") - public static final NoOpUnaryUdfOp INSTANCE = new NoOpUnaryUdfOp(); - - public NoOpUnaryUdfOp() { - // pass null here because we override getOutputType to return type - // of input operator - super(new UserCodeClassWrapper(NoOpFunction.class), null, ""); - } - - @Override - public UnaryOperatorInformation getOperatorInfo() { - TypeInformation previousOut = input.getOperatorInfo().getOutputType(); - return new UnaryOperatorInformation(previousOut, previousOut); - } - - @Override - protected List executeOnCollections( - List inputData, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { - return inputData; - } -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/Utils.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/Utils.java deleted file mode 100644 index af9d2197f9722..0000000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/util/Utils.java +++ /dev/null @@ -1,131 +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.optimizer.util; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.AtomicType; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.CompositeType; -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypeComparatorFactory; -import org.apache.flink.api.java.typeutils.runtime.RuntimeComparatorFactory; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.PlanNode; - -import java.util.Arrays; - -/** Utility class that contains helper methods for optimizer. */ -public final class Utils { - - public static FieldList createOrderedFromSet(FieldSet set) { - if (set instanceof FieldList) { - return (FieldList) set; - } else { - final int[] cols = set.toArray(); - Arrays.sort(cols); - return new FieldList(cols); - } - } - - public static Ordering createOrdering(FieldList fields, boolean[] directions) { - final Ordering o = new Ordering(); - for (int i = 0; i < fields.size(); i++) { - o.appendOrdering( - fields.get(i), - null, - directions == null || directions[i] ? Order.ASCENDING : Order.DESCENDING); - } - return o; - } - - public static Ordering createOrdering(FieldList fields) { - final Ordering o = new Ordering(); - for (int i = 0; i < fields.size(); i++) { - o.appendOrdering(fields.get(i), null, Order.ANY); - } - return o; - } - - public static boolean[] getDirections(Ordering o, int numFields) { - final boolean[] dirs = o.getFieldSortDirections(); - if (dirs.length == numFields) { - return dirs; - } else if (dirs.length > numFields) { - final boolean[] subSet = new boolean[numFields]; - System.arraycopy(dirs, 0, subSet, 0, numFields); - return subSet; - } else { - throw new CompilerException(); - } - } - - public static TypeComparatorFactory getShipComparator( - Channel channel, ExecutionConfig executionConfig) { - PlanNode source = channel.getSource(); - Operator javaOp = source.getProgramOperator(); - TypeInformation type = javaOp.getOperatorInfo().getOutputType(); - return createComparator( - type, - channel.getShipStrategyKeys(), - getSortOrders(channel.getShipStrategyKeys(), channel.getShipStrategySortOrder()), - executionConfig); - } - - private static TypeComparatorFactory createComparator( - TypeInformation typeInfo, - FieldList keys, - boolean[] sortOrder, - ExecutionConfig executionConfig) { - - TypeComparator comparator; - if (typeInfo instanceof CompositeType) { - comparator = - ((CompositeType) typeInfo) - .createComparator(keys.toArray(), sortOrder, 0, executionConfig); - } else if (typeInfo instanceof AtomicType) { - // handle grouping of atomic types - comparator = ((AtomicType) typeInfo).createComparator(sortOrder[0], executionConfig); - } else { - throw new RuntimeException("Unrecognized type: " + typeInfo); - } - - return new RuntimeComparatorFactory<>(comparator); - } - - private static boolean[] getSortOrders(FieldList keys, boolean[] orders) { - if (orders == null) { - orders = new boolean[keys.size()]; - Arrays.fill(orders, true); - } - return orders; - } - - // -------------------------------------------------------------------------------------------- - - /** No instantiation. */ - private Utils() { - throw new RuntimeException(); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java deleted file mode 100644 index dfcf6a09ab734..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java +++ /dev/null @@ -1,101 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** - * Tests that validate optimizer choices when using operators that are requesting certain specific - * execution strategies. - */ -@SuppressWarnings({"serial"}) -public class AdditionalOperatorsTest extends CompilerTestBase { - - @Test - public void testCrossWithSmall() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet set1 = env.generateSequence(0, 1); - DataSet set2 = env.generateSequence(0, 1); - - set1.crossWithTiny(set2) - .name("Cross") - .output(new DiscardingOutputFormat>()); - - try { - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - OptimizerPlanNodeResolver resolver = new OptimizerPlanNodeResolver(oPlan); - - DualInputPlanNode crossPlanNode = resolver.getNode("Cross"); - Channel in1 = crossPlanNode.getInput1(); - Channel in2 = crossPlanNode.getInput2(); - - assertEquals(ShipStrategyType.FORWARD, in1.getShipStrategy()); - assertEquals(ShipStrategyType.BROADCAST, in2.getShipStrategy()); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The Flink optimizer is unable to compile this plan correctly."); - } - } - - @Test - public void testCrossWithLarge() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet set1 = env.generateSequence(0, 1); - DataSet set2 = env.generateSequence(0, 1); - - set1.crossWithHuge(set2) - .name("Cross") - .output(new DiscardingOutputFormat>()); - - try { - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - OptimizerPlanNodeResolver resolver = new OptimizerPlanNodeResolver(oPlan); - - DualInputPlanNode crossPlanNode = resolver.getNode("Cross"); - Channel in1 = crossPlanNode.getInput1(); - Channel in2 = crossPlanNode.getInput2(); - - assertEquals(ShipStrategyType.BROADCAST, in1.getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, in2.getShipStrategy()); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly."); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java deleted file mode 100644 index cc55d1d5b50c4..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java +++ /dev/null @@ -1,1068 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.RichJoinFunction; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.io.TextOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.optimizer.testfunctions.IdentityCoGrouper; -import org.apache.flink.optimizer.testfunctions.IdentityCrosser; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.testfunctions.IdentityJoiner; -import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.testfunctions.Top1GroupReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashSet; -import java.util.Set; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial"}) -public class BranchingPlansCompilerTest extends CompilerTestBase { - - @Test - public void testCostComputationWithMultipleDataSinks() { - final int SINKS = 5; - - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet source = env.generateSequence(1, 10000); - - DataSet mappedA = source.map(new IdentityMapper()); - DataSet mappedC = source.map(new IdentityMapper()); - - for (int sink = 0; sink < SINKS; sink++) { - mappedA.output(new DiscardingOutputFormat()); - mappedC.output(new DiscardingOutputFormat()); - } - - Plan plan = env.createProgramPlan("Plans With Multiple Data Sinks"); - OptimizedPlan oPlan = compileNoStats(plan); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * - * - *

-     *                (SRC A)
-     *                   |
-     *                (MAP A)
-     *             /         \
-     *          (MAP B)      (MAP C)
-     *           /           /     \
-     *        (SINK A)    (SINK B)  (SINK C)
-     * 
- */ - @SuppressWarnings("unchecked") - @Test - public void testBranchingWithMultipleDataSinks2() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet source = env.generateSequence(1, 10000); - - DataSet mappedA = source.map(new IdentityMapper()); - DataSet mappedB = mappedA.map(new IdentityMapper()); - DataSet mappedC = mappedA.map(new IdentityMapper()); - - mappedB.output(new DiscardingOutputFormat()); - mappedC.output(new DiscardingOutputFormat()); - mappedC.output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - Set> sinks = new HashSet>(plan.getDataSinks()); - - OptimizedPlan oPlan = compileNoStats(plan); - - // ---------- check the optimizer plan ---------- - - // number of sinks - assertEquals("Wrong number of data sinks.", 3, oPlan.getDataSinks().size()); - - // remove matching sinks to check relation - for (SinkPlanNode sink : oPlan.getDataSinks()) { - assertTrue(sinks.remove(sink.getProgramOperator())); - } - assertTrue(sinks.isEmpty()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * - * - *
-     *                              SINK
-     *                               |
-     *                            COGROUP
-     *                        +---/    \----+
-     *                       /               \
-     *                      /             MATCH10
-     *                     /               |    \
-     *                    /                |  MATCH9
-     *                MATCH5               |  |   \
-     *                |   \                |  | MATCH8
-     *                | MATCH4             |  |  |   \
-     *                |  |   \             |  |  | MATCH7
-     *                |  | MATCH3          |  |  |  |   \
-     *                |  |  |   \          |  |  |  | MATCH6
-     *                |  |  | MATCH2       |  |  |  |  |  |
-     *                |  |  |  |   \       +--+--+--+--+--+
-     *                |  |  |  | MATCH1            MAP
-     *                \  |  |  |  |  | /-----------/
-     *                (DATA SOURCE ONE)
-     * 
- */ - @Test - public void testBranchingSourceMultipleTimes() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> source = - env.generateSequence(1, 10000000).map(new Duplicator()); - - DataSet> joined1 = - source.join(source) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined2 = - source.join(joined1) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined3 = - source.join(joined2) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined4 = - source.join(joined3) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined5 = - source.join(joined4) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> mapped = - source.map( - new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple2 value) { - return null; - } - }); - - DataSet> joined6 = - mapped.join(mapped) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined7 = - mapped.join(joined6) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined8 = - mapped.join(joined7) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined9 = - mapped.join(joined8) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined10 = - mapped.join(joined9) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - joined5.coGroup(joined10) - .where(1) - .equalTo(1) - .with(new DummyCoGroupFunction, Tuple2>()) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * - * - *
-     *
-     *              (SINK A)
-     *                  |    (SINK B)    (SINK C)
-     *                CROSS    /          /
-     *               /     \   |  +------+
-     *              /       \  | /
-     *          REDUCE      MATCH2
-     *             |    +---/    \
-     *              \  /          |
-     *               MAP          |
-     *                |           |
-     *             COGROUP      MATCH1
-     *             /     \     /     \
-     *        (SRC A)    (SRC B)    (SRC C)
-     * 
- */ - @Test - public void testBranchingWithMultipleDataSinks() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> sourceA = - env.generateSequence(1, 10000000).map(new Duplicator()); - - DataSet> sourceB = - env.generateSequence(1, 10000000).map(new Duplicator()); - - DataSet> sourceC = - env.generateSequence(1, 10000000).map(new Duplicator()); - - DataSet> mapped = - sourceA.coGroup(sourceB) - .where(0) - .equalTo(1) - .with( - new CoGroupFunction< - Tuple2, - Tuple2, - Tuple2>() { - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) {} - }) - .map(new IdentityMapper>()); - - DataSet> joined = - sourceB.join(sourceC) - .where(0) - .equalTo(1) - .with(new DummyFlatJoinFunction>()); - - DataSet> joined2 = - mapped.join(joined) - .where(1) - .equalTo(1) - .with(new DummyFlatJoinFunction>()); - - DataSet> reduced = - mapped.groupBy(1).reduceGroup(new Top1GroupReducer>()); - - reduced.cross(joined2) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()); - - joined2.output(new DiscardingOutputFormat>()); - joined2.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @SuppressWarnings("unchecked") - @Test - public void testBranchEachContractType() { - try { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet sourceA = env.generateSequence(0, 1); - DataSet sourceB = env.generateSequence(0, 1); - DataSet sourceC = env.generateSequence(0, 1); - - DataSet map1 = sourceA.map(new IdentityMapper()).name("Map 1"); - - DataSet reduce1 = - map1.groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .name("Reduce 1"); - - DataSet join1 = - sourceB.union(sourceB) - .union(sourceC) - .join(sourceC) - .where("*") - .equalTo("*") - .with(new IdentityJoiner()) - .name("Join 1"); - - DataSet coGroup1 = - sourceA.coGroup(sourceB) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 1"); - - DataSet cross1 = - reduce1.cross(coGroup1).with(new IdentityCrosser()).name("Cross 1"); - - DataSet coGroup2 = - cross1.coGroup(cross1) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 2"); - - DataSet coGroup3 = - map1.coGroup(join1) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 3"); - - DataSet map2 = coGroup3.map(new IdentityMapper()).name("Map 2"); - - DataSet coGroup4 = - map2.coGroup(join1) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 4"); - - DataSet coGroup5 = - coGroup2.coGroup(coGroup1) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 5"); - - DataSet coGroup6 = - reduce1.coGroup(coGroup4) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 6"); - - DataSet coGroup7 = - coGroup5.coGroup(coGroup6) - .where("*") - .equalTo("*") - .with(new IdentityCoGrouper()) - .name("CoGroup 7"); - - coGroup7.union(sourceA) - .union(coGroup3) - .union(coGroup4) - .union(coGroup1) - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - - JobGraphGenerator jobGen = new JobGraphGenerator(); - - // Compile plan to verify that no error is thrown - jobGen.compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test - public void testBranchingUnion() { - try { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet source1 = env.generateSequence(0, 1); - DataSet source2 = env.generateSequence(0, 1); - - DataSet join1 = - source1.join(source2) - .where("*") - .equalTo("*") - .with(new IdentityJoiner()) - .name("Join 1"); - - DataSet map1 = join1.map(new IdentityMapper()).name("Map 1"); - - DataSet reduce1 = - map1.groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .name("Reduce 1"); - - DataSet reduce2 = - join1.groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .name("Reduce 2"); - - DataSet map2 = join1.map(new IdentityMapper()).name("Map 2"); - - DataSet map3 = map2.map(new IdentityMapper()).name("Map 3"); - - DataSet join2 = - reduce1.union(reduce2) - .union(map2) - .union(map3) - .join(map2, JoinHint.REPARTITION_SORT_MERGE) - .where("*") - .equalTo("*") - .with(new IdentityJoiner()) - .name("Join 2"); - - join2.output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - - JobGraphGenerator jobGen = new JobGraphGenerator(); - - // Compile plan to verify that no error is thrown - jobGen.compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * - * - *
-     *             (SRC A)
-     *             /     \
-     *        (SINK A)    (SINK B)
-     * 
- */ - @Test - public void testBranchingWithMultipleDataSinksSmall() { - try { - String outPath1 = "/tmp/out1"; - String outPath2 = "/tmp/out2"; - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet source1 = env.generateSequence(0, 1); - - source1.writeAsText(outPath1); - source1.writeAsText(outPath2); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - - // ---------- check the optimizer plan ---------- - - // number of sinks - Assert.assertEquals("Wrong number of data sinks.", 2, oPlan.getDataSinks().size()); - - // sinks contain all sink paths - Set allSinks = new HashSet(); - allSinks.add(outPath1); - allSinks.add(outPath2); - - for (SinkPlanNode n : oPlan.getDataSinks()) { - String path = - ((TextOutputFormat) - n.getSinkNode() - .getOperator() - .getFormatWrapper() - .getUserCodeObject()) - .getOutputFilePath() - .toString(); - Assert.assertTrue("Invalid data sink.", allSinks.remove(path)); - } - - // ---------- compile plan to job graph to verify that no error is thrown ---------- - - JobGraphGenerator jobGen = new JobGraphGenerator(); - jobGen.compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * - * - *
-     *     (SINK 3) (SINK 1)   (SINK 2) (SINK 4)
-     *         \     /             \     /
-     *         (SRC A)             (SRC B)
-     * 
- * - * NOTE: this case is currently not caught by the compiler. we should enable the test once it is - * caught. - */ - @Test - public void testBranchingDisjointPlan() { - // construct the plan - final String out1Path = "file:///test/1"; - final String out2Path = "file:///test/2"; - final String out3Path = "file:///test/3"; - final String out4Path = "file:///test/4"; - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet sourceA = env.generateSequence(0, 1); - DataSet sourceB = env.generateSequence(0, 1); - - sourceA.writeAsText(out1Path); - sourceB.writeAsText(out2Path); - sourceA.writeAsText(out3Path); - sourceB.writeAsText(out4Path); - - Plan plan = env.createProgramPlan(); - compileNoStats(plan); - } - - @Test - public void testBranchAfterIteration() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet sourceA = env.generateSequence(0, 1); - - IterativeDataSet loopHead = sourceA.iterate(10); - DataSet loopTail = loopHead.map(new IdentityMapper()).name("Mapper"); - DataSet loopRes = loopHead.closeWith(loopTail); - - loopRes.output(new DiscardingOutputFormat()); - loopRes.map(new IdentityMapper()).output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test - public void testBranchBeforeIteration() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet source1 = env.generateSequence(0, 1); - DataSet source2 = env.generateSequence(0, 1); - - IterativeDataSet loopHead = source2.iterate(10).name("Loop"); - DataSet loopTail = - source1.map(new IdentityMapper()) - .withBroadcastSet(loopHead, "BC") - .name("In-Loop Mapper"); - DataSet loopRes = loopHead.closeWith(loopTail); - - DataSet map = - source1.map(new IdentityMapper()) - .withBroadcastSet(loopRes, "BC") - .name("Post-Loop Mapper"); - map.output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * Test to ensure that sourceA is inside as well as outside of the iteration the same node. - * - *
-     *       (SRC A)               (SRC B)
-     *      /       \             /       \
-     *  (SINK 1)   (ITERATION)    |     (SINK 2)
-     *             /        \     /
-     *         (SINK 3)     (CROSS => NEXT PARTIAL SOLUTION)
-     * 
- */ - @Test - public void testClosure() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet sourceA = env.generateSequence(0, 1); - DataSet sourceB = env.generateSequence(0, 1); - - sourceA.output(new DiscardingOutputFormat()); - sourceB.output(new DiscardingOutputFormat()); - - IterativeDataSet loopHead = sourceA.iterate(10).name("Loop"); - - DataSet loopTail = loopHead.cross(sourceB).with(new IdentityCrosser()); - DataSet loopRes = loopHead.closeWith(loopTail); - - loopRes.output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * - * - *
-     *       (SRC A)         (SRC B)          (SRC C)
-     *      /       \       /                /       \
-     *  (SINK 1) (DELTA ITERATION)          |     (SINK 2)
-     *             /    |   \               /
-     *         (SINK 3) |   (CROSS => NEXT WORKSET)
-     *                  |             |
-     *                (JOIN => SOLUTION SET DELTA)
-     * 
- */ - @Test - public void testClosureDeltaIteration() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet> sourceA = - env.generateSequence(0, 1).map(new Duplicator()); - DataSet> sourceB = - env.generateSequence(0, 1).map(new Duplicator()); - DataSet> sourceC = - env.generateSequence(0, 1).map(new Duplicator()); - - sourceA.output(new DiscardingOutputFormat>()); - sourceC.output(new DiscardingOutputFormat>()); - - DeltaIteration, Tuple2> loop = - sourceA.iterateDelta(sourceB, 10, 0); - - DataSet> workset = - loop.getWorkset() - .cross(sourceB) - .with(new IdentityCrosser>()) - .name("Next work set"); - DataSet> delta = - workset.join(loop.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new IdentityJoiner>()) - .name("Solution set delta"); - - DataSet> result = loop.closeWith(delta, workset); - result.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * - * - *
-     *                  +----Iteration-------+
-     *                  |                    |
-     *       /---------< >---------join-----< >---sink
-     *      / (Solution)|           /        |
-     *     /            |          /         |
-     *    /--map-------< >----\   /       /--|
-     *   /     (Workset)|      \ /       /   |
-     * src-map          |     join------/    |
-     *   \              |      /             |
-     *    \             +-----/--------------+
-     *     \                 /
-     *      \--reduce-------/
-     * 
- */ - @Test - public void testDeltaIterationWithStaticInput() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet> source = env.generateSequence(0, 1).map(new Duplicator()); - - DataSet> map = source.map(new IdentityMapper>()); - DataSet> reduce = - source.reduceGroup(new IdentityGroupReducer>()); - - DeltaIteration, Tuple2> loop = - source.iterateDelta(map, 10, 0); - - DataSet> workset = - loop.getWorkset() - .join(reduce) - .where(0) - .equalTo(0) - .with(new IdentityJoiner>()) - .name("Next work set"); - DataSet> delta = - loop.getSolutionSet() - .join(workset) - .where(0) - .equalTo(0) - .with(new IdentityJoiner>()) - .name("Solution set delta"); - - DataSet> result = loop.closeWith(delta, workset); - result.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * - * - *
-     *             +---------Iteration-------+
-     *             |                         |
-     *    /--map--< >----\                   |
-     *   /         |      \         /-------< >---sink
-     * src-map     |     join------/         |
-     *   \         |      /                  |
-     *    \        +-----/-------------------+
-     *     \            /
-     *      \--reduce--/
-     * 
- */ - @Test - public void testIterationWithStaticInput() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(100); - - DataSet source = env.generateSequence(1, 1000000); - - DataSet mapped = source.map(new IdentityMapper()); - - DataSet reduced = - source.groupBy(new IdentityKeyExtractor()) - .reduce(new SelectOneReducer()); - - IterativeDataSet iteration = mapped.iterate(10); - iteration - .closeWith( - iteration - .join(reduced) - .where(new IdentityKeyExtractor()) - .equalTo(new IdentityKeyExtractor()) - .with(new DummyFlatJoinFunction())) - .output(new DiscardingOutputFormat()); - - compileNoStats(env.createProgramPlan()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testBranchingBroadcastVariable() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(100); - - DataSet input1 = env.readTextFile(IN_FILE).name("source1"); - DataSet input2 = env.readTextFile(IN_FILE).name("source2"); - DataSet input3 = env.readTextFile(IN_FILE).name("source3"); - - DataSet result1 = - input1.map(new IdentityMapper()) - .reduceGroup(new Top1GroupReducer()) - .withBroadcastSet(input3, "bc"); - - DataSet result2 = - input2.map(new IdentityMapper()) - .reduceGroup(new Top1GroupReducer()) - .withBroadcastSet(input3, "bc"); - - result1.join(result2) - .where(new IdentityKeyExtractor()) - .equalTo(new IdentityKeyExtractor()) - .with( - new RichJoinFunction() { - @Override - public String join(String first, String second) { - return null; - } - }) - .withBroadcastSet(input3, "bc1") - .withBroadcastSet(input1, "bc2") - .withBroadcastSet(result1, "bc3") - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test - public void testBCVariableClosure() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input = env.readTextFile(IN_FILE).name("source1"); - - DataSet reduced = - input.map(new IdentityMapper()).reduceGroup(new Top1GroupReducer()); - - DataSet initialSolution = - input.map(new IdentityMapper()).withBroadcastSet(reduced, "bc"); - - IterativeDataSet iteration = initialSolution.iterate(100); - - iteration - .closeWith( - iteration - .map(new IdentityMapper()) - .withBroadcastSet(reduced, "red")) - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test - public void testMultipleIterations() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(100); - - DataSet input = env.readTextFile(IN_FILE).name("source1"); - - DataSet reduced = - input.map(new IdentityMapper()).reduceGroup(new Top1GroupReducer()); - - IterativeDataSet iteration1 = input.iterate(100); - IterativeDataSet iteration2 = input.iterate(20); - IterativeDataSet iteration3 = input.iterate(17); - - iteration1 - .closeWith( - iteration1 - .map(new IdentityMapper()) - .withBroadcastSet(reduced, "bc1")) - .output(new DiscardingOutputFormat()); - iteration2 - .closeWith( - iteration2 - .reduceGroup(new Top1GroupReducer()) - .withBroadcastSet(reduced, "bc2")) - .output(new DiscardingOutputFormat()); - iteration3 - .closeWith( - iteration3 - .reduceGroup(new IdentityGroupReducer()) - .withBroadcastSet(reduced, "bc3")) - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test - public void testMultipleIterationsWithClosueBCVars() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(100); - - DataSet input = env.readTextFile(IN_FILE).name("source1"); - - IterativeDataSet iteration1 = input.iterate(100); - IterativeDataSet iteration2 = input.iterate(20); - IterativeDataSet iteration3 = input.iterate(17); - - iteration1 - .closeWith(iteration1.map(new IdentityMapper())) - .output(new DiscardingOutputFormat()); - iteration2 - .closeWith(iteration2.reduceGroup(new Top1GroupReducer())) - .output(new DiscardingOutputFormat()); - iteration3 - .closeWith(iteration3.reduceGroup(new IdentityGroupReducer())) - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - try { - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test - public void testBranchesOnlyInBCVariables1() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(100); - - DataSet input = env.generateSequence(1, 10); - DataSet bc_input = env.generateSequence(1, 10); - - input.map(new IdentityMapper()) - .withBroadcastSet(bc_input, "name1") - .map(new IdentityMapper()) - .withBroadcastSet(bc_input, "name2") - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testBranchesOnlyInBCVariables2() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(100); - - DataSet> input = - env.generateSequence(1, 10).map(new Duplicator()).name("proper input"); - - DataSet bc_input1 = env.generateSequence(1, 10).name("BC input 1"); - DataSet bc_input2 = env.generateSequence(1, 10).name("BC input 1"); - - DataSet> joinInput1 = - input.map(new IdentityMapper>()) - .withBroadcastSet(bc_input1.map(new IdentityMapper()), "bc1") - .withBroadcastSet(bc_input2, "bc2"); - - DataSet> joinInput2 = - input.map(new IdentityMapper>()) - .withBroadcastSet(bc_input1, "bc1") - .withBroadcastSet(bc_input2, "bc2"); - - DataSet> joinResult = - joinInput1 - .join(joinInput2, JoinHint.REPARTITION_HASH_FIRST) - .where(0) - .equalTo(1) - .with(new DummyFlatJoinFunction>()); - - input.map(new IdentityMapper>()) - .withBroadcastSet(bc_input1, "bc1") - .union(joinResult) - .output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - compileNoStats(plan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private static final class Duplicator implements MapFunction> { - - @Override - public Tuple2 map(T value) { - return new Tuple2(value, value); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java deleted file mode 100644 index 2c7410b874c3c..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BroadcastVariablePipelinebreakerTest.java +++ /dev/null @@ -1,104 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class BroadcastVariablePipelinebreakerTest extends CompilerTestBase { - - @Test - public void testNoBreakerForIndependentVariable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet source1 = env.fromElements("test"); - DataSet source2 = env.fromElements("test"); - - source1.map(new IdentityMapper()) - .withBroadcastSet(source2, "some name") - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, mapper.getBroadcastInputs().get(0).getTempMode()); - - assertEquals(DataExchangeMode.PIPELINED, mapper.getInput().getDataExchangeMode()); - assertEquals( - DataExchangeMode.PIPELINED, - mapper.getBroadcastInputs().get(0).getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testBreakerForDependentVariable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet source1 = env.fromElements("test"); - - source1.map(new IdentityMapper()) - .map(new IdentityMapper()) - .withBroadcastSet(source1, "some name") - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode beforeMapper = (SingleInputPlanNode) mapper.getInput().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, beforeMapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, mapper.getBroadcastInputs().get(0).getTempMode()); - - assertEquals(DataExchangeMode.PIPELINED, mapper.getInput().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, beforeMapper.getInput().getDataExchangeMode()); - assertEquals( - DataExchangeMode.BATCH, - mapper.getBroadcastInputs().get(0).getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java deleted file mode 100644 index 94abe5335d2df..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CachedMatchStrategyCompilerTest.java +++ /dev/null @@ -1,292 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichJoinFunction; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -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.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** Tests that validate optimizer choice when using hash joins inside of iterations */ -@SuppressWarnings("serial") -public class CachedMatchStrategyCompilerTest extends CompilerTestBase { - - /** - * This tests whether a HYBRIDHASH_BUILD_SECOND is correctly transformed to a - * HYBRIDHASH_BUILD_SECOND_CACHED when inside of an iteration an on the static path - */ - @Test - public void testRightSide() { - try { - - Plan plan = getTestPlanRightStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); - - // verify correct join strategy - assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED, innerJoin.getDriverStrategy()); - assertEquals(TempMode.NONE, innerJoin.getInput1().getTempMode()); - assertEquals(TempMode.NONE, innerJoin.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - /** - * This test makes sure that only a HYBRIDHASH on the static path is transformed to the cached - * variant - */ - @Test - public void testRightSideCountercheck() { - try { - - Plan plan = getTestPlanRightStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); - - // verify correct join strategy - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, innerJoin.getDriverStrategy()); - assertEquals(TempMode.NONE, innerJoin.getInput1().getTempMode()); - assertEquals(TempMode.CACHED, innerJoin.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - /** - * This tests whether a HYBRIDHASH_BUILD_FIRST is correctly transformed to a - * HYBRIDHASH_BUILD_FIRST_CACHED when inside of an iteration an on the static path - */ - @Test - public void testLeftSide() { - try { - - Plan plan = getTestPlanLeftStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_FIRST); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); - - // verify correct join strategy - assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_FIRST_CACHED, innerJoin.getDriverStrategy()); - assertEquals(TempMode.NONE, innerJoin.getInput1().getTempMode()); - assertEquals(TempMode.NONE, innerJoin.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - /** - * This test makes sure that only a HYBRIDHASH on the static path is transformed to the cached - * variant - */ - @Test - public void testLeftSideCountercheck() { - try { - - Plan plan = getTestPlanLeftStatic(Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); - - // verify correct join strategy - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, innerJoin.getDriverStrategy()); - assertEquals(TempMode.CACHED, innerJoin.getInput1().getTempMode()); - assertEquals(TempMode.NONE, innerJoin.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - /** - * This test simulates a join of a big left side with a small right side inside of an iteration, - * where the small side is on a static path. Currently the best execution plan is a - * HYBRIDHASH_BUILD_SECOND_CACHED, where the small side is hashed and cached. This test also - * makes sure that all relevant plans are correctly enumerated by the optimizer. - */ - @Test - public void testCorrectChoosing() { - try { - - Plan plan = getTestPlanRightStatic(""); - - SourceCollectorVisitor sourceCollector = new SourceCollectorVisitor(); - plan.accept(sourceCollector); - - for (GenericDataSourceBase s : sourceCollector.getSources()) { - if (s.getName().equals("bigFile")) { - this.setSourceStatistics(s, 10000000, 1000); - } else if (s.getName().equals("smallFile")) { - this.setSourceStatistics(s, 100, 100); - } - } - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode innerJoin = resolver.getNode("DummyJoiner"); - - // verify correct join strategy - assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED, innerJoin.getDriverStrategy()); - assertEquals(TempMode.NONE, innerJoin.getInput1().getTempMode()); - assertEquals(TempMode.NONE, innerJoin.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - private Plan getTestPlanRightStatic(String strategy) { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> bigInput = - env.readCsvFile("file://bigFile") - .types(Long.class, Long.class, Long.class) - .name("bigFile"); - - DataSet> smallInput = - env.readCsvFile("file://smallFile") - .types(Long.class, Long.class, Long.class) - .name("smallFile"); - - IterativeDataSet> iteration = bigInput.iterate(10); - - Configuration joinStrategy = new Configuration(); - joinStrategy.setString( - Optimizer.HINT_SHIP_STRATEGY, Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH); - - if (!strategy.equals("")) { - joinStrategy.setString(Optimizer.HINT_LOCAL_STRATEGY, strategy); - } - - DataSet> inner = - iteration - .join(smallInput) - .where(0) - .equalTo(0) - .with(new DummyJoiner()) - .name("DummyJoiner") - .withParameters(joinStrategy); - - DataSet> output = iteration.closeWith(inner); - - output.output(new DiscardingOutputFormat>()); - - return env.createProgramPlan(); - } - - private Plan getTestPlanLeftStatic(String strategy) { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - @SuppressWarnings("unchecked") - DataSet> bigInput = - env.fromElements( - new Tuple3(1L, 2L, 3L), - new Tuple3(1L, 2L, 3L), - new Tuple3(1L, 2L, 3L)) - .name("Big"); - - @SuppressWarnings("unchecked") - DataSet> smallInput = - env.fromElements(new Tuple3(1L, 2L, 3L)).name("Small"); - - IterativeDataSet> iteration = bigInput.iterate(10); - - Configuration joinStrategy = new Configuration(); - joinStrategy.setString(Optimizer.HINT_LOCAL_STRATEGY, strategy); - - DataSet> inner = - smallInput - .join(iteration) - .where(0) - .equalTo(0) - .with(new DummyJoiner()) - .name("DummyJoiner") - .withParameters(joinStrategy); - - DataSet> output = iteration.closeWith(inner); - - output.output(new DiscardingOutputFormat>()); - - return env.createProgramPlan(); - } - - private static class DummyJoiner - extends RichJoinFunction< - Tuple3, Tuple3, Tuple3> { - - @Override - public Tuple3 join( - Tuple3 first, Tuple3 second) throws Exception { - - return first; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java deleted file mode 100644 index fc36ffc2b7919..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/CoGroupSolutionSetFirstTest.java +++ /dev/null @@ -1,124 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichCoGroupFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; -import org.apache.flink.util.Visitor; - -import org.junit.Assert; -import org.junit.Test; - -@SuppressWarnings("serial") -public class CoGroupSolutionSetFirstTest extends CompilerTestBase { - - public static class SimpleCGroup - extends RichCoGroupFunction, Tuple1, Tuple1> { - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) {} - } - - public static class SimpleMap extends RichMapFunction, Tuple1> { - @Override - public Tuple1 map(Tuple1 value) throws Exception { - return null; - } - } - - @Test - public void testCoGroupSolutionSet() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> raw = env.readCsvFile(IN_FILE).types(Integer.class); - - DeltaIteration, Tuple1> iteration = raw.iterateDelta(raw, 1000, 0); - - DataSet> test = iteration.getWorkset().map(new SimpleMap()); - DataSet> delta = - iteration - .getSolutionSet() - .coGroup(test) - .where(0) - .equalTo(0) - .with(new SimpleCGroup()); - DataSet> feedback = iteration.getWorkset().map(new SimpleMap()); - DataSet> result = iteration.closeWith(delta, feedback); - - result.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = null; - try { - oPlan = compileNoStats(plan); - } catch (CompilerException e) { - Assert.fail(e.getMessage()); - } - - oPlan.accept( - new Visitor() { - @Override - public boolean preVisit(PlanNode visitable) { - if (visitable instanceof WorksetIterationPlanNode) { - PlanNode deltaNode = - ((WorksetIterationPlanNode) visitable) - .getSolutionSetDeltaPlanNode(); - - // get the CoGroup - DualInputPlanNode dpn = - (DualInputPlanNode) - deltaNode.getInputs().iterator().next().getSource(); - Channel in1 = dpn.getInput1(); - Channel in2 = dpn.getInput2(); - - Assert.assertTrue(in1.getLocalProperties().getOrdering() == null); - Assert.assertTrue(in2.getLocalProperties().getOrdering() != null); - Assert.assertTrue( - in2.getLocalProperties() - .getOrdering() - .getInvolvedIndexes() - .contains(0)); - Assert.assertTrue(in1.getShipStrategy() == ShipStrategyType.FORWARD); - Assert.assertTrue( - in2.getShipStrategy() == ShipStrategyType.PARTITION_HASH); - return false; - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) {} - }); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java deleted file mode 100644 index d922f12891909..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DisjointDataFlowsTest.java +++ /dev/null @@ -1,53 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class DisjointDataFlowsTest extends CompilerTestBase { - - @Test - public void testDisjointFlows() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // generate two different flows - env.generateSequence(1, 10).output(new DiscardingOutputFormat()); - env.generateSequence(1, 10).output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java deleted file mode 100644 index 734b02d593b93..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/DistinctCompilationTest.java +++ /dev/null @@ -1,281 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint; -import org.apache.flink.api.common.operators.util.FieldList; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DistinctOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -@SuppressWarnings("serial") -public class DistinctCompilationTest extends CompilerTestBase implements java.io.Serializable { - - @Test - public void testDistinctPlain() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.distinct() - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0, 1), reduceNode.getKeys(0)); - assertEquals(new FieldList(0, 1), combineNode.getKeys(0)); - assertEquals(new FieldList(0, 1), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testDistinctWithCombineHint() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.distinct() - .setCombineHint(CombineHint.HASH) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.HASHED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0, 1), reduceNode.getKeys(0)); - assertEquals(new FieldList(0, 1), combineNode.getKeys(0)); - assertEquals(new FieldList(0, 1), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testDistinctWithSelectorFunctionKey() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.distinct( - new KeySelector, String>() { - public String getKey(Tuple2 value) { - return value.f0; - } - }) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // get the key extractors and projectors - SingleInputPlanNode keyExtractor = - (SingleInputPlanNode) combineNode.getInput().getSource(); - SingleInputPlanNode keyProjector = - (SingleInputPlanNode) sinkNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, keyExtractor.getInput().getSource()); - assertEquals(keyProjector, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0), reduceNode.getKeys(0)); - assertEquals(new FieldList(0), combineNode.getKeys(0)); - assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, keyExtractor.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, keyProjector.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testDistinctWithFieldPositionKeyCombinable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - DistinctOperator> reduced = data.distinct(1).name("reducer"); - - reduced.output(new DiscardingOutputFormat>()).name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(1), reduceNode.getKeys(0)); - assertEquals(new FieldList(1), combineNode.getKeys(0)); - assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java deleted file mode 100644 index 3b6a66eabd139..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/FeedbackPropertiesMatchTest.java +++ /dev/null @@ -1,1602 +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.optimizer; - -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.BinaryOperatorInformation; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.OperatorInformation; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.UnaryOperatorInformation; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.dag.DataSourceNode; -import org.apache.flink.optimizer.dag.JoinNode; -import org.apache.flink.optimizer.dag.MapNode; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.junit.Test; - -import static org.apache.flink.optimizer.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport.*; -import static org.junit.Assert.*; - -public class FeedbackPropertiesMatchTest { - - @Test - public void testNoPartialSolutionFoundSingleInputOnly() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Source"); - - SourcePlanNode otherTarget = new SourcePlanNode(getSourceNode(), "Source"); - - Channel toMap1 = new Channel(target); - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(map1); - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = new LocalProperties(); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(otherTarget, gp, lp); - assertTrue(report == NO_PARTIAL_SOLUTION); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testSingleInputOperators() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Source"); - - Channel toMap1 = new Channel(target); - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(map1); - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - // no feedback properties and none are ever required and present - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = new LocalProperties(); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some global feedback properties and none are ever required and present - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 5)); - LocalProperties lp = new LocalProperties(); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some local feedback properties and none are ever required and present - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some global and local feedback properties and none are ever required and present - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 5)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // --------------------------- requirements on channel 1 ----------------------- - - // some required global properties, which are matched exactly - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 5)); - LocalProperties lp = new LocalProperties(); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldList(2, 5)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required local properties, which are matched exactly - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1, 2)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global and local properties, which are matched exactly - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 5)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldList(2, 5)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1, 2)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global and local properties, which are over-fulfilled - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldSet(2, 5)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global properties that are not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 1)); - LocalProperties lp = new LocalProperties(); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldList(2, 5)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some required local properties that are not met - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2, 1)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some required global and local properties where the global properties are not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 1)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(2, 5)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some required global and local properties where the local properties are not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(1)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // --------------------------- requirements on channel 2 ----------------------- - - // some required global properties, which are matched exactly - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 5)); - LocalProperties lp = new LocalProperties(); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldList(2, 5)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required local properties, which are matched exactly - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1, 2)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global and local properties, which are matched exactly - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 5)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldList(2, 5)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1, 2)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global and local properties, which are over-fulfilled - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1, 2)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldSet(2, 5)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global properties that are not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 1)); - LocalProperties lp = new LocalProperties(); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setHashPartitioned(new FieldSet(2, 5)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some required local properties that are not met - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2, 1)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some required global and local properties where the global properties are not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(2, 1)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldSet(2, 5)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(1)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some required global and local properties where the local properties are not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(1)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // ---------------------- requirements mixed on 1 and 2 ----------------------- - - // some required global properties at step one and some more at step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.EMPTY; - - RequestedGlobalProperties reqGp1 = new RequestedGlobalProperties(); - reqGp1.setAnyPartitioning(new FieldList(1, 2)); - - RequestedGlobalProperties reqGp2 = new RequestedGlobalProperties(); - reqGp2.setHashPartitioned(new FieldList(1, 2)); - - toMap1.setRequiredGlobalProps(reqGp1); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp2); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required local properties at step one and some more at step 2 - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = - LocalProperties.forOrdering( - new Ordering(3, null, Order.ASCENDING) - .appendOrdering(1, null, Order.DESCENDING)); - - RequestedLocalProperties reqLp1 = new RequestedLocalProperties(); - reqLp1.setGroupedFields(new FieldList(3, 1)); - - RequestedLocalProperties reqLp2 = new RequestedLocalProperties(); - reqLp2.setOrdering( - new Ordering(3, null, Order.ANY).appendOrdering(1, null, Order.ANY)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp1); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp2); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required global properties at step one and some local ones at step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(1, 2)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some required local properties at step one and some global ones at step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(1, 2)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some fulfilled global properties at step one and some non-fulfilled local ones at - // step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(1, 2)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2, 3)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some fulfilled local properties at step one and some non-fulfilled global ones at - // step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(2, 3)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2, 1)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some non-fulfilled global properties at step one and some fulfilled local ones at - // step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(2, 3)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2, 1)); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // some non-fulfilled local properties at step one and some fulfilled global ones at - // step 2 - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldList(1, 2)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(2, 1, 3)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testSingleInputOperatorsWithReCreation() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Source"); - - Channel toMap1 = new Channel(target); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(map1); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - // set ship strategy in first channel, so later non matching global properties do not - // matter - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.EMPTY; - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldSet(2, 5)); - - toMap1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(2, 5), - DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(reqGp); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(MET, report); - } - - // set ship strategy in second channel, so previous non matching global properties void - // the match - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = LocalProperties.EMPTY; - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldSet(2, 5)); - - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - - toMap2.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(2, 5), - DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // set local strategy in first channel, so later non matching local properties do not - // matter - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = - LocalProperties.forOrdering( - new Ordering(3, null, Order.ASCENDING) - .appendOrdering(1, null, Order.DESCENDING)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(4, 1)); - - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false}); - - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // set local strategy in second channel, so previous non matching local properties void - // the match - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = - LocalProperties.forOrdering( - new Ordering(3, null, Order.ASCENDING) - .appendOrdering(1, null, Order.DESCENDING)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(4, 1)); - - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy( - LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false}); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // create the properties on the same node as the requirement - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(1, 2)); - LocalProperties lp = - LocalProperties.forOrdering( - new Ordering(3, null, Order.ASCENDING) - .appendOrdering(1, null, Order.DESCENDING)); - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldSet(5, 7)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(5, 7)); - - toMap1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(5, 7), - DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false}); - - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - - toMap1.setRequiredGlobalProps(reqGp); - toMap1.setRequiredLocalProps(reqLp); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map2.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(MET, report); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testSingleInputOperatorsChainOfThree() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Source"); - - Channel toMap1 = new Channel(target); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(map1); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - Channel toMap3 = new Channel(map2); - SingleInputPlanNode map3 = - new SingleInputPlanNode(getMapNode(), "Mapper 3", toMap3, DriverStrategy.MAP); - - // set local strategy in first channel, so later non matching local properties do not - // matter - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = - LocalProperties.forOrdering( - new Ordering(3, null, Order.ASCENDING) - .appendOrdering(1, null, Order.DESCENDING)); - - RequestedLocalProperties reqLp = new RequestedLocalProperties(); - reqLp.setGroupedFields(new FieldList(4, 1)); - - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(5, 7), new boolean[] {false, false}); - - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - - toMap3.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap3.setLocalStrategy(LocalStrategy.NONE); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - toMap3.setRequiredGlobalProps(null); - toMap3.setRequiredLocalProps(reqLp); - - FeedbackPropertiesMeetRequirementsReport report = - map3.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // set global strategy in first channel, so later non matching global properties do not - // matter - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(5, 3)); - LocalProperties lp = LocalProperties.EMPTY; - - RequestedGlobalProperties reqGp = new RequestedGlobalProperties(); - reqGp.setAnyPartitioning(new FieldSet(2, 3)); - - toMap1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(1, 2), - DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - - toMap3.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap3.setLocalStrategy(LocalStrategy.NONE); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toMap2.setRequiredGlobalProps(null); - toMap2.setRequiredLocalProps(null); - - toMap3.setRequiredGlobalProps(reqGp); - toMap3.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - map3.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testNoPartialSolutionFoundTwoInputOperator() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Partial Solution"); - - SourcePlanNode source1 = new SourcePlanNode(getSourceNode(), "Source 1"); - SourcePlanNode source2 = new SourcePlanNode(getSourceNode(), "Source 2"); - - Channel toMap1 = new Channel(source1); - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(source2); - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - Channel toJoin1 = new Channel(map1); - Channel toJoin2 = new Channel(map2); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin2.setLocalStrategy(LocalStrategy.NONE); - - DualInputPlanNode join = - new DualInputPlanNode( - getJoinNode(), - "Join", - toJoin1, - toJoin2, - DriverStrategy.HYBRIDHASH_BUILD_FIRST); - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet( - target, new GlobalProperties(), new LocalProperties()); - assertEquals(NO_PARTIAL_SOLUTION, report); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testTwoOperatorsOneIndependent() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Partial Solution"); - SourcePlanNode source = new SourcePlanNode(getSourceNode(), "Other Source"); - - Channel toMap1 = new Channel(target); - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(source); - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - Channel toJoin1 = new Channel(map1); - Channel toJoin2 = new Channel(map2); - - DualInputPlanNode join = - new DualInputPlanNode( - getJoinNode(), - "Join", - toJoin1, - toJoin2, - DriverStrategy.HYBRIDHASH_BUILD_FIRST); - - Channel toAfterJoin = new Channel(join); - toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toAfterJoin.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode afterJoin = - new SingleInputPlanNode( - getMapNode(), "After Join Mapper", toAfterJoin, DriverStrategy.MAP); - - // attach some properties to the non-relevant input - { - toMap2.setShipStrategy(ShipStrategyType.BROADCAST, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy( - LocalStrategy.SORT, new FieldList(2, 7), new boolean[] {true, true}); - - RequestedGlobalProperties joinGp = new RequestedGlobalProperties(); - joinGp.setFullyReplicated(); - - RequestedLocalProperties joinLp = new RequestedLocalProperties(); - joinLp.setOrdering( - new Ordering(2, null, Order.ASCENDING) - .appendOrdering(7, null, Order.ASCENDING)); - - toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin2.setLocalStrategy(LocalStrategy.NONE); - toJoin2.setRequiredGlobalProps(joinGp); - toJoin2.setRequiredLocalProps(joinLp); - } - - // ------------------------------------------------------------------------------------ - - // no properties from the partial solution, no required properties - { - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.EMPTY; - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some properties from the partial solution, no required properties - { - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // produced properties match relevant input - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(2)); - - toJoin1.setRequiredGlobalProps(rgp); - toJoin1.setRequiredLocalProps(rlp); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // produced properties do not match relevant input - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(1, 2, 3)); - - toJoin1.setRequiredGlobalProps(rgp); - toJoin1.setRequiredLocalProps(rlp); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // produced properties overridden before join - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(2, 1)); - - toMap1.setRequiredGlobalProps(rgp); - toMap1.setRequiredLocalProps(rlp); - - toJoin1.setRequiredGlobalProps(null); - toJoin1.setRequiredLocalProps(null); - - toJoin1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(2, 1), - DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false}); - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(MET, report); - } - - // produced properties before join match, after join match as well - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(2, 1)); - - toMap1.setRequiredGlobalProps(null); - toMap1.setRequiredLocalProps(null); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - toJoin1.setRequiredGlobalProps(rgp); - toJoin1.setRequiredLocalProps(rlp); - - toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toAfterJoin.setLocalStrategy(LocalStrategy.NONE); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - join.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // produced properties before join match, after join do not match - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp1 = new RequestedGlobalProperties(); - rgp1.setHashPartitioned(new FieldList(0)); - - RequestedGlobalProperties rgp2 = new RequestedGlobalProperties(); - rgp2.setHashPartitioned(new FieldList(3)); - - RequestedLocalProperties rlp1 = new RequestedLocalProperties(); - rlp1.setGroupedFields(new FieldList(2, 1)); - - RequestedLocalProperties rlp2 = new RequestedLocalProperties(); - rlp2.setGroupedFields(new FieldList(3, 4)); - - toJoin1.setRequiredGlobalProps(rgp1); - toJoin1.setRequiredLocalProps(rlp1); - - toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toAfterJoin.setLocalStrategy(LocalStrategy.NONE); - - toAfterJoin.setRequiredGlobalProps(rgp2); - toAfterJoin.setRequiredLocalProps(rlp2); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // produced properties are overridden, does not matter that they do not match - { - GlobalProperties gp = new GlobalProperties(); - gp.setAnyPartitioning(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(1)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(1, 2, 3)); - - toJoin1.setRequiredGlobalProps(null); - toJoin1.setRequiredLocalProps(null); - - toJoin1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(2, 1), - DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false}); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(MET, report); - } - - // local property overridden before join, local property mismatch after join not - // relevant - { - GlobalProperties gp = new GlobalProperties(); - gp.setAnyPartitioning(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(1, 2, 3)); - - toJoin1.setRequiredGlobalProps(null); - toJoin1.setRequiredLocalProps(null); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false}); - - toAfterJoin.setRequiredGlobalProps(null); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // local property overridden before join, global property mismatch after join void the - // match - { - GlobalProperties gp = new GlobalProperties(); - gp.setAnyPartitioning(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(1)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(1, 2, 3)); - - toJoin1.setRequiredGlobalProps(null); - toJoin1.setRequiredLocalProps(null); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(7, 3), new boolean[] {true, false}); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testTwoOperatorsBothDependent() { - try { - SourcePlanNode target = new SourcePlanNode(getSourceNode(), "Partial Solution"); - - Channel toMap1 = new Channel(target); - toMap1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap1.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map1 = - new SingleInputPlanNode(getMapNode(), "Mapper 1", toMap1, DriverStrategy.MAP); - - Channel toMap2 = new Channel(target); - toMap2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toMap2.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode map2 = - new SingleInputPlanNode(getMapNode(), "Mapper 2", toMap2, DriverStrategy.MAP); - - Channel toJoin1 = new Channel(map1); - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - Channel toJoin2 = new Channel(map2); - toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin2.setLocalStrategy(LocalStrategy.NONE); - - DualInputPlanNode join = - new DualInputPlanNode( - getJoinNode(), - "Join", - toJoin1, - toJoin2, - DriverStrategy.HYBRIDHASH_BUILD_FIRST); - - Channel toAfterJoin = new Channel(join); - toAfterJoin.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toAfterJoin.setLocalStrategy(LocalStrategy.NONE); - SingleInputPlanNode afterJoin = - new SingleInputPlanNode( - getMapNode(), "After Join Mapper", toAfterJoin, DriverStrategy.MAP); - - // no properties from the partial solution, no required properties - { - GlobalProperties gp = new GlobalProperties(); - LocalProperties lp = LocalProperties.EMPTY; - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // some properties from the partial solution, no required properties - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // test requirements on one input and met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(2, 1)); - - toJoin1.setRequiredGlobalProps(rgp); - toJoin1.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // test requirements on both input and met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(2, 1)); - - toJoin1.setRequiredGlobalProps(rgp); - toJoin1.setRequiredLocalProps(rlp); - - toJoin2.setRequiredGlobalProps(rgp); - toJoin2.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertTrue(report != null && report != NO_PARTIAL_SOLUTION && report != NOT_MET); - } - - // test requirements on both inputs, one not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp1 = new RequestedGlobalProperties(); - rgp1.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp1 = new RequestedLocalProperties(); - rlp1.setGroupedFields(new FieldList(2, 1)); - - RequestedGlobalProperties rgp2 = new RequestedGlobalProperties(); - rgp2.setHashPartitioned(new FieldList(1)); - - RequestedLocalProperties rlp2 = new RequestedLocalProperties(); - rlp2.setGroupedFields(new FieldList(0, 3)); - - toJoin1.setRequiredGlobalProps(rgp1); - toJoin1.setRequiredLocalProps(rlp1); - - toJoin2.setRequiredGlobalProps(rgp2); - toJoin2.setRequiredLocalProps(rlp2); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // test override on both inputs, later requirement ignored - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(1)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(0, 3)); - - toJoin1.setRequiredGlobalProps(null); - toJoin1.setRequiredLocalProps(null); - - toJoin2.setRequiredGlobalProps(null); - toJoin2.setRequiredLocalProps(null); - - toJoin1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(88), - DataExchangeMode.PIPELINED); - toJoin2.setShipStrategy(ShipStrategyType.BROADCAST, DataExchangeMode.PIPELINED); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(MET, report); - } - - // test override on one inputs, later requirement met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(0)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(2, 1)); - - toJoin1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(88), - DataExchangeMode.PIPELINED); - toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(PENDING, report); - } - - // test override on one input, later requirement not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(3)); - - RequestedLocalProperties rlp = new RequestedLocalProperties(); - rlp.setGroupedFields(new FieldList(77, 69)); - - toJoin1.setShipStrategy( - ShipStrategyType.PARTITION_HASH, - new FieldList(88), - DataExchangeMode.PIPELINED); - toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(rlp); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - - // test override on one input locally, later global requirement not met - { - GlobalProperties gp = new GlobalProperties(); - gp.setHashPartitioned(new FieldList(0)); - LocalProperties lp = LocalProperties.forGrouping(new FieldList(2, 1)); - - RequestedGlobalProperties rgp = new RequestedGlobalProperties(); - rgp.setHashPartitioned(new FieldList(3)); - - toJoin1.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy( - LocalStrategy.SORT, new FieldList(3), new boolean[] {false}); - - toJoin2.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED); - toJoin1.setLocalStrategy(LocalStrategy.NONE); - - toAfterJoin.setRequiredGlobalProps(rgp); - toAfterJoin.setRequiredLocalProps(null); - - FeedbackPropertiesMeetRequirementsReport report = - afterJoin.checkPartialSolutionPropertiesMet(target, gp, lp); - assertEquals(NOT_MET, report); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static DataSourceNode getSourceNode() { - return new DataSourceNode( - new GenericDataSourceBase( - new TextInputFormat(new Path("/")), - new OperatorInformation(BasicTypeInfo.STRING_TYPE_INFO))); - } - - private static MapNode getMapNode() { - return new MapNode( - new MapOperatorBase>( - new IdentityMapper(), - new UnaryOperatorInformation( - BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO), - "map op")); - } - - private static JoinNode getJoinNode() { - return new JoinNode( - new InnerJoinOperatorBase< - String, String, String, FlatJoinFunction>( - new DummyFlatJoinFunction(), - new BinaryOperatorInformation( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO), - new int[] {1}, - new int[] {2}, - "join op")); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.java deleted file mode 100644 index 63a829b4755be..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/GroupOrderTest.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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.util.FieldList; -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.Tuple4; -import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityCoGrouper; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.junit.Assert; -import org.junit.Test; - -import static org.junit.Assert.fail; - -/** - * This test case has been created to validate that correct strategies are used if orders within - * groups are requested. - */ -@SuppressWarnings({"serial"}) -public class GroupOrderTest extends CompilerTestBase { - - @Test - public void testReduceWithGroupOrder() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet> set1 = - env.readCsvFile("/tmp/fake.csv") - .types(Long.class, Long.class, Long.class, Long.class); - - set1.groupBy(1) - .sortGroup(3, Order.DESCENDING) - .reduceGroup(new IdentityGroupReducer>()) - .name("Reduce") - .output(new DiscardingOutputFormat>()) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan; - - try { - oPlan = compileNoStats(plan); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly."); - return; // silence the compiler - } - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - SinkPlanNode sinkNode = resolver.getNode("Sink"); - SingleInputPlanNode reducer = resolver.getNode("Reduce"); - - // verify the strategies - Assert.assertEquals(ShipStrategyType.FORWARD, sinkNode.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); - - Channel c = reducer.getInput(); - Assert.assertEquals(LocalStrategy.SORT, c.getLocalStrategy()); - - FieldList ship = new FieldList(1); - FieldList local = new FieldList(1, 3); - Assert.assertEquals(ship, c.getShipStrategyKeys()); - Assert.assertEquals(local, c.getLocalStrategyKeys()); - Assert.assertTrue(c.getLocalStrategySortOrder()[0] == reducer.getSortOrders(0)[0]); - - // check that we indeed sort descending - Assert.assertEquals(false, c.getLocalStrategySortOrder()[1]); - } - - @Test - public void testCoGroupWithGroupOrder() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet> set1 = - env.readCsvFile("/tmp/fake1.csv") - .types( - Long.class, - Long.class, - Long.class, - Long.class, - Long.class, - Long.class, - Long.class); - DataSet> set2 = - env.readCsvFile("/tmp/fake2.csv") - .types( - Long.class, - Long.class, - Long.class, - Long.class, - Long.class, - Long.class, - Long.class); - - set1.coGroup(set2) - .where(3, 0) - .equalTo(6, 0) - .sortFirstGroup(5, Order.DESCENDING) - .sortSecondGroup(1, Order.DESCENDING) - .sortSecondGroup(4, Order.ASCENDING) - .with(new IdentityCoGrouper>()) - .name("CoGroup") - .output( - new DiscardingOutputFormat< - Tuple7>()) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan; - - try { - oPlan = compileNoStats(plan); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly."); - return; // silence the compiler - } - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - SinkPlanNode sinkNode = resolver.getNode("Sink"); - DualInputPlanNode coGroupNode = resolver.getNode("CoGroup"); - - // verify the strategies - Assert.assertEquals(ShipStrategyType.FORWARD, sinkNode.getInput().getShipStrategy()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, coGroupNode.getInput1().getShipStrategy()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, coGroupNode.getInput2().getShipStrategy()); - - Channel c1 = coGroupNode.getInput1(); - Channel c2 = coGroupNode.getInput2(); - - Assert.assertEquals(LocalStrategy.SORT, c1.getLocalStrategy()); - Assert.assertEquals(LocalStrategy.SORT, c2.getLocalStrategy()); - - FieldList ship1 = new FieldList(3, 0); - FieldList ship2 = new FieldList(6, 0); - - FieldList local1 = new FieldList(3, 0, 5); - FieldList local2 = new FieldList(6, 0, 1, 4); - - Assert.assertEquals(ship1, c1.getShipStrategyKeys()); - Assert.assertEquals(ship2, c2.getShipStrategyKeys()); - Assert.assertEquals(local1, c1.getLocalStrategyKeys()); - Assert.assertEquals(local2, c2.getLocalStrategyKeys()); - - Assert.assertTrue(c1.getLocalStrategySortOrder()[0] == coGroupNode.getSortOrders()[0]); - Assert.assertTrue(c1.getLocalStrategySortOrder()[1] == coGroupNode.getSortOrders()[1]); - Assert.assertTrue(c2.getLocalStrategySortOrder()[0] == coGroupNode.getSortOrders()[0]); - Assert.assertTrue(c2.getLocalStrategySortOrder()[1] == coGroupNode.getSortOrders()[1]); - - // check that the local group orderings are correct - Assert.assertEquals(false, c1.getLocalStrategySortOrder()[2]); - Assert.assertEquals(false, c2.getLocalStrategySortOrder()[2]); - Assert.assertEquals(true, c2.getLocalStrategySortOrder()[3]); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java deleted file mode 100644 index 43a125371e9fd..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/HardPlansCompilationTest.java +++ /dev/null @@ -1,84 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityCrosser; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -/** - * This class tests plans that once failed because of a bug: - * - *
    - *
  • Ticket 158 - *
- */ -@SuppressWarnings({"serial"}) -public class HardPlansCompilationTest extends CompilerTestBase { - - /** - * Source -> Map -> Reduce -> Cross -> Reduce -> Cross -> Reduce -> |--------------------------/ - * / |--------------------------------------------/ - * - *

First cross has SameKeyFirst output contract - */ - @Test - public void testTicket158() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet set1 = env.generateSequence(0, 1); - - set1.map(new IdentityMapper()) - .name("Map1") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .name("Reduce1") - .cross(set1) - .with(new IdentityCrosser()) - .withForwardedFieldsFirst("*") - .name("Cross1") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .name("Reduce2") - .cross(set1) - .with(new IdentityCrosser()) - .name("Cross2") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .name("Reduce3") - .output(new DiscardingOutputFormat()) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - - JobGraphGenerator jobGen = new JobGraphGenerator(); - jobGen.compileJobGraph(oPlan); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java deleted file mode 100644 index b6d4a9025a73c..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/IterationsCompilerTest.java +++ /dev/null @@ -1,632 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.*; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.functions.FunctionAnnotation.ForwardedFields; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; - -import org.junit.Test; - -import java.util.Iterator; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class IterationsCompilerTest extends CompilerTestBase { - - @Test - public void testSolutionSetDeltaDependsOnBroadcastVariable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> source = - env.generateSequence(1, 1000).map(new DuplicateValueScalar()); - - DataSet> invariantInput = - env.generateSequence(1, 1000).map(new DuplicateValueScalar()); - - // iteration from here - DeltaIteration, Tuple2> iter = - source.iterateDelta(source, 1000, 1); - - DataSet> result = - invariantInput - .map(new IdentityMapper>()) - .withBroadcastSet(iter.getWorkset(), "bc data") - .join(iter.getSolutionSet()) - .where(0) - .equalTo(1) - .projectFirst(1) - .projectSecond(1); - - iter.closeWith(result.map(new IdentityMapper>()), result) - .output(new DiscardingOutputFormat>()); - - OptimizedPlan p = compileNoStats(env.createProgramPlan()); - - // check that the JSON generator accepts this plan - new PlanJSONDumpGenerator().getOptimizerPlanAsJSON(p); - - // check that the JobGraphGenerator accepts the plan - new JobGraphGenerator().compileJobGraph(p); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testTwoIterationsWithMapperInbetween() throws Exception { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> verticesWithInitialId = - env.fromElements(new Tuple2(1L, 2L)); - - DataSet> edges = env.fromElements(new Tuple2(1L, 2L)); - - DataSet> bulkResult = doBulkIteration(verticesWithInitialId, edges); - - DataSet> mappedBulk = bulkResult.map(new DummyMap()); - - DataSet> depResult = doDeltaIteration(mappedBulk, edges); - - depResult.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(1, op.getDataSinks().size()); - assertTrue( - op.getDataSinks().iterator().next().getInput().getSource() - instanceof WorksetIterationPlanNode); - - WorksetIterationPlanNode wipn = - (WorksetIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_HASH, wipn.getInput1().getShipStrategy()); - - assertEquals(TempMode.NONE, wipn.getInput1().getTempMode()); - assertEquals(TempMode.NONE, wipn.getInput2().getTempMode()); - - assertEquals(DataExchangeMode.BATCH, wipn.getInput1().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, wipn.getInput2().getDataExchangeMode()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testTwoIterationsDirectlyChained() throws Exception { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> verticesWithInitialId = - env.fromElements(new Tuple2(1L, 2L)); - - DataSet> edges = env.fromElements(new Tuple2(1L, 2L)); - - DataSet> bulkResult = doBulkIteration(verticesWithInitialId, edges); - - DataSet> depResult = doDeltaIteration(bulkResult, edges); - - depResult.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(1, op.getDataSinks().size()); - assertTrue( - op.getDataSinks().iterator().next().getInput().getSource() - instanceof WorksetIterationPlanNode); - - WorksetIterationPlanNode wipn = - (WorksetIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - BulkIterationPlanNode bipn = (BulkIterationPlanNode) wipn.getInput1().getSource(); - - // the hash partitioning has been pushed out of the delta iteration into the bulk - // iteration - assertEquals(ShipStrategyType.FORWARD, wipn.getInput1().getShipStrategy()); - - // the input of the root step function is the last operator of the step function - // since the work has been pushed out of the bulk iteration, it has to guarantee the - // hash partitioning - for (Channel c : bipn.getRootOfStepFunction().getInputs()) { - assertEquals(ShipStrategyType.PARTITION_HASH, c.getShipStrategy()); - } - - assertEquals(DataExchangeMode.BATCH, wipn.getInput1().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, wipn.getInput2().getDataExchangeMode()); - - assertEquals(TempMode.NONE, wipn.getInput1().getTempMode()); - assertEquals(TempMode.NONE, wipn.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testTwoWorksetIterationsDirectlyChained() throws Exception { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> verticesWithInitialId = - env.fromElements(new Tuple2(1L, 2L)); - - DataSet> edges = env.fromElements(new Tuple2(1L, 2L)); - - DataSet> firstResult = - doDeltaIteration(verticesWithInitialId, edges); - - DataSet> secondResult = doDeltaIteration(firstResult, edges); - - secondResult.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(1, op.getDataSinks().size()); - assertTrue( - op.getDataSinks().iterator().next().getInput().getSource() - instanceof WorksetIterationPlanNode); - - WorksetIterationPlanNode wipn = - (WorksetIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, wipn.getInput1().getShipStrategy()); - - assertEquals(DataExchangeMode.BATCH, wipn.getInput1().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, wipn.getInput2().getDataExchangeMode()); - - assertEquals(TempMode.NONE, wipn.getInput1().getTempMode()); - assertEquals(TempMode.NONE, wipn.getInput2().getTempMode()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testIterationPushingWorkOut() throws Exception { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> input1 = - env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue()); - - DataSet> input2 = - env.readCsvFile("/some/file/path").types(Long.class, Long.class); - - // we do two join operations with input1 which is the partial solution - // it is cheaper to push the partitioning out so that the feedback channel and the - // initial input do the partitioning - doBulkIteration(input1, input2) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(1, op.getDataSinks().size()); - assertTrue( - op.getDataSinks().iterator().next().getInput().getSource() - instanceof BulkIterationPlanNode); - - BulkIterationPlanNode bipn = - (BulkIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - - // check that work has been pushed out - for (Channel c : bipn.getPartialSolutionPlanNode().getOutgoingChannels()) { - assertEquals(ShipStrategyType.FORWARD, c.getShipStrategy()); - } - - // the end of the step function has to produce the necessary properties - for (Channel c : bipn.getRootOfStepFunction().getInputs()) { - assertEquals(ShipStrategyType.PARTITION_HASH, c.getShipStrategy()); - } - - assertEquals(ShipStrategyType.PARTITION_HASH, bipn.getInput().getShipStrategy()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testIterationNotPushingWorkOut() throws Exception { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> input1 = - env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue()); - - DataSet> input2 = - env.readCsvFile("/some/file/path").types(Long.class, Long.class); - - // Use input1 as partial solution. Partial solution is used in a single join operation - // --> it is cheaper - // to do the hash partitioning between the partial solution node and the join node - // instead of pushing the partitioning out - doSimpleBulkIteration(input1, input2) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(1, op.getDataSinks().size()); - assertTrue( - op.getDataSinks().iterator().next().getInput().getSource() - instanceof BulkIterationPlanNode); - - BulkIterationPlanNode bipn = - (BulkIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - - // check that work has not been pushed out - for (Channel c : bipn.getPartialSolutionPlanNode().getOutgoingChannels()) { - assertEquals(ShipStrategyType.PARTITION_HASH, c.getShipStrategy()); - } - - assertEquals(ShipStrategyType.FORWARD, bipn.getInput().getShipStrategy()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testWorksetIterationPipelineBreakerPlacement() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - // the workset (input two of the delta iteration) is the same as what is consumed be the - // successive join - DataSet> initialWorkset = - env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue()); - - DataSet> initialSolutionSet = - env.readCsvFile("/some/file/path").types(Long.class).map(new DuplicateValue()); - - // trivial iteration, since we are interested in the inputs to the iteration - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialWorkset, 100, 0); - - DataSet> next = - iteration.getWorkset().map(new IdentityMapper>()); - - DataSet> result = iteration.closeWith(next, next); - - initialWorkset - .join(result, JoinHint.REPARTITION_HASH_FIRST) - .where(0) - .equalTo(0) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()); - - Plan p = env.createProgramPlan(); - compileNoStats(p); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testResetPartialSolution() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet width = env.generateSequence(1, 10); - DataSet update = env.generateSequence(1, 10); - DataSet lastGradient = env.generateSequence(1, 10); - - DataSet init = width.union(update).union(lastGradient); - - IterativeDataSet iteration = init.iterate(10); - - width = iteration.filter(new IdFilter()); - update = iteration.filter(new IdFilter()); - lastGradient = iteration.filter(new IdFilter()); - - DataSet gradient = width.map(new IdentityMapper()); - DataSet term = - gradient.join(lastGradient) - .where(new IdentityKeyExtractor()) - .equalTo(new IdentityKeyExtractor()) - .with( - new JoinFunction() { - public Long join(Long first, Long second) { - return null; - } - }); - - update = - update.map( - new RichMapFunction() { - public Long map(Long value) { - return null; - } - }) - .withBroadcastSet(term, "some-name"); - - DataSet result = iteration.closeWith(width.union(update).union(lastGradient)); - - result.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * Tests that interesting properties can be pushed out of the bulk iteration. This requires that - * a NoOp node is appended to the step function which re-establishes the properties of the - * initial input. If this does not work, then Flink won't find a plan, because the optimizer - * will not consider plans where the partitioning is done after the partial solution node in - * this case (because of pruning). - * - * @throws Exception - */ - @Test - public void testBulkIterationWithPartialSolutionProperties() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = - env.generateSequence(1, 10) - .map( - new MapFunction>() { - @Override - public Tuple1 map(Long value) throws Exception { - return new Tuple1<>(value); - } - }); - - DataSet> input2 = - env.generateSequence(1, 10) - .map( - new MapFunction>() { - @Override - public Tuple1 map(Long value) throws Exception { - return new Tuple1<>(value); - } - }); - - DataSet> distinctInput = input1.distinct(); - - IterativeDataSet> iteration = distinctInput.iterate(10); - - DataSet> iterationStep = - iteration - .coGroup(input2) - .where(0) - .equalTo(0) - .with( - new CoGroupFunction, Tuple1, Tuple1>() { - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception { - Iterator> it = first.iterator(); - - if (it.hasNext()) { - out.collect(it.next()); - } - } - }); - - DataSet> iterationResult = iteration.closeWith(iterationStep); - - iterationResult.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - new JobGraphGenerator().compileJobGraph(op); - } - - // -------------------------------------------------------------------------------------------- - - public static DataSet> doBulkIteration( - DataSet> vertices, DataSet> edges) { - - // open a bulk iteration - IterativeDataSet> iteration = vertices.iterate(20); - - DataSet> changes = - iteration - .join(edges) - .where(0) - .equalTo(0) - .with(new Join222()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration) - .where(0) - .equalTo(0) - .flatMap(new FlatMapJoin()); - - // close the bulk iteration - return iteration.closeWith(changes); - } - - public static DataSet> doSimpleBulkIteration( - DataSet> vertices, DataSet> edges) { - - // open a bulk iteration - IterativeDataSet> iteration = vertices.iterate(20); - - DataSet> changes = - iteration.join(edges).where(0).equalTo(0).flatMap(new FlatMapJoin()); - - // close the bulk iteration - return iteration.closeWith(changes); - } - - public static DataSet> doDeltaIteration( - DataSet> vertices, DataSet> edges) { - - DeltaIteration, Tuple2> depIteration = - vertices.iterateDelta(vertices, 100, 0); - - DataSet> candidates = - depIteration.getWorkset().join(edges).where(0).equalTo(0).projectSecond(1); - - DataSet> grouped = candidates.groupBy(0).reduceGroup(new Reduce101()); - - DataSet> candidatesDependencies = - grouped.join(edges).where(0).equalTo(1).projectSecond(0, 1); - - DataSet> verticesWithNewComponents = - candidatesDependencies - .join(depIteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new Join222()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1); - - DataSet> updatedComponentId = - verticesWithNewComponents - .join(depIteration.getSolutionSet()) - .where(0) - .equalTo(0) - .flatMap(new FlatMapJoin()); - - DataSet> depResult = - depIteration.closeWith(updatedComponentId, updatedComponentId); - - return depResult; - } - - // -------------------------------------------------------------------------------------------- - - public static final class Join222 - extends RichJoinFunction, Tuple2, Tuple2> { - - @Override - public Tuple2 join( - Tuple2 vertexWithComponent, Tuple2 edge) { - return null; - } - } - - public static final class FlatMapJoin - extends RichFlatMapFunction< - Tuple2, Tuple2>, Tuple2> { - - @Override - public void flatMap( - Tuple2, Tuple2> value, - Collector> out) {} - } - - public static final class DummyMap - extends RichMapFunction, Tuple2> { - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - return value; - } - } - - @ForwardedFields("0") - public static final class Reduce101 - extends RichGroupReduceFunction, Tuple1> { - - @Override - public void reduce(Iterable> values, Collector> out) {} - } - - @ForwardedFields("0") - public static final class DuplicateValue - extends RichMapFunction, Tuple2> { - - @Override - public Tuple2 map(Tuple1 value) throws Exception { - return new Tuple2(value.f0, value.f0); - } - } - - public static final class DuplicateValueScalar extends RichMapFunction> { - - @Override - public Tuple2 map(T value) { - return new Tuple2(value, value); - } - } - - public static final class IdFilter implements FilterFunction { - @Override - public boolean filter(T value) { - return true; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java deleted file mode 100644 index 2eeac66384f76..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/NestedIterationsTest.java +++ /dev/null @@ -1,204 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class NestedIterationsTest extends CompilerTestBase { - - @Test - public void testRejectNestedBulkIterations() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.generateSequence(1, 100); - - IterativeDataSet outerIteration = data.iterate(100); - - IterativeDataSet innerIteration = - outerIteration.map(new IdentityMapper()).iterate(100); - - DataSet innerResult = - innerIteration.closeWith(innerIteration.map(new IdentityMapper())); - - DataSet outerResult = - outerIteration.closeWith(innerResult.map(new IdentityMapper())); - - outerResult.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - - try { - compileNoStats(p); - } catch (CompilerException e) { - assertTrue(e.getMessage().toLowerCase().indexOf("nested iterations") != -1); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testRejectNestedWorksetIterations() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = env.fromElements(new Tuple2(0L, 0L)); - - DeltaIteration, Tuple2> outerIteration = - data.iterateDelta(data, 100, 0); - - DataSet> inOuter = - outerIteration.getWorkset().map(new IdentityMapper>()); - - DeltaIteration, Tuple2> innerIteration = - inOuter.iterateDelta(inOuter, 100, 0); - - DataSet> inInner = - innerIteration.getWorkset().map(new IdentityMapper>()); - - DataSet> innerResult = - innerIteration - .closeWith(inInner, inInner) - .map(new IdentityMapper>()); - - DataSet> outerResult = - outerIteration.closeWith(innerResult, innerResult); - - outerResult.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - - try { - compileNoStats(p); - } catch (CompilerException e) { - assertTrue(e.getMessage().toLowerCase().indexOf("nested iterations") != -1); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testBulkIterationInClosure() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data1 = env.generateSequence(1, 100); - DataSet data2 = env.generateSequence(1, 100); - - IterativeDataSet firstIteration = data1.iterate(100); - - DataSet firstResult = - firstIteration.closeWith(firstIteration.map(new IdentityMapper())); - - IterativeDataSet mainIteration = - data2.map(new IdentityMapper()).iterate(100); - - DataSet joined = - mainIteration - .join(firstResult) - .where(new IdentityKeyExtractor()) - .equalTo(new IdentityKeyExtractor()) - .with(new DummyFlatJoinFunction()); - - DataSet mainResult = mainIteration.closeWith(joined); - - mainResult.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - - // optimizer should be able to translate this - OptimizedPlan op = compileNoStats(p); - - // job graph generator should be able to translate this - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testDeltaIterationInClosure() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> data2 = env.fromElements(new Tuple2(0L, 0L)); - - DeltaIteration, Tuple2> firstIteration = - data1.iterateDelta(data1, 100, 0); - - DataSet> inFirst = - firstIteration.getWorkset().map(new IdentityMapper>()); - - DataSet> firstResult = - firstIteration - .closeWith(inFirst, inFirst) - .map(new IdentityMapper>()); - - DeltaIteration, Tuple2> mainIteration = - data2.iterateDelta(data2, 100, 0); - - DataSet> joined = - mainIteration - .getWorkset() - .join(firstResult) - .where(0) - .equalTo(0) - .projectFirst(0) - .projectSecond(0); - - DataSet> mainResult = mainIteration.closeWith(joined, joined); - - mainResult.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - - // optimizer should be able to translate this - OptimizedPlan op = compileNoStats(p); - - // job graph generator should be able to translate this - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java deleted file mode 100644 index d9edcda25f2ad..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ParallelismChangeTest.java +++ /dev/null @@ -1,381 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.testfunctions.IdentityJoiner; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Visitor; - -import org.junit.Assert; -import org.junit.Test; - -/** - * Tests in this class: - * - *

    - *
  • Tests that check the correct handling of the properties and strategies in the case where - * the parallelism between tasks is increased or decreased. - *
- */ -@SuppressWarnings({"serial"}) -public class ParallelismChangeTest extends CompilerTestBase { - - /** - * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all - * properties). - * - *

Increases parallelism between 1st reduce and 2nd map, so the hash partitioning from 1st - * reduce is not reusable. Expected to re-establish partitioning between reduce and map, via - * hash, because random is a full network transit as well. - */ - @Test - public void checkPropertyHandlingWithIncreasingGlobalParallelism1() { - final int p = DEFAULT_PARALLELISM; - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(p); - DataSet set1 = env.generateSequence(0, 1).setParallelism(p); - - set1.map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p) - .name("Map1") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p) - .name("Reduce1") - .map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Map2") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Reduce2") - .output(new DiscardingOutputFormat()) - .setParallelism(p * 2) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when reducer 1 distributes its data across the instances of map2, it needs to employ a - // local hash method, - // because map2 has twice as many instances and key/value pairs with the same key need to be - // processed by the same - // mapper respectively reducer - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor(); - SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor(); - - ShipStrategyType mapIn = map2Node.getInput().getShipStrategy(); - ShipStrategyType redIn = red2Node.getInput().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.PARTITION_HASH, mapIn); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, redIn); - } - - /** - * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all - * properties). - * - *

Increases parallelism between 2nd map and 2nd reduce, so the hash partitioning from 1st - * reduce is not reusable. Expected to re-establish partitioning between map and reduce (hash). - */ - @Test - public void checkPropertyHandlingWithIncreasingGlobalParallelism2() { - final int p = DEFAULT_PARALLELISM; - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(p); - DataSet set1 = env.generateSequence(0, 1).setParallelism(p); - - set1.map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p) - .name("Map1") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p) - .name("Reduce1") - .map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p) - .name("Map2") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Reduce2") - .output(new DiscardingOutputFormat()) - .setParallelism(p * 2) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when reducer 1 distributes its data across the instances of map2, it needs to employ a - // local hash method, - // because map2 has twice as many instances and key/value pairs with the same key need to be - // processed by the same - // mapper respectively reducer - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor(); - SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor(); - - ShipStrategyType mapIn = map2Node.getInput().getShipStrategy(); - ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, mapIn); - Assert.assertEquals( - "Invalid ship strategy for an operator.", - ShipStrategyType.PARTITION_HASH, - reduceIn); - } - - /** - * Simple Job: Map -> Reduce -> Map -> Reduce. All functions preserve all fields (hence all - * properties). - * - *

Increases parallelism between 1st reduce and 2nd map, such that more tasks are on one - * instance. Expected to re-establish partitioning between map and reduce via a local hash. - */ - @Test - public void checkPropertyHandlingWithIncreasingLocalParallelism() { - final int p = DEFAULT_PARALLELISM * 2; - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(p); - DataSet set1 = env.generateSequence(0, 1).setParallelism(p); - - set1.map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p) - .name("Map1") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p) - .name("Reduce1") - .map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Map2") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Reduce2") - .output(new DiscardingOutputFormat()) - .setParallelism(p * 2) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when reducer 1 distributes its data across the instances of map2, it needs to employ a - // local hash method, - // because map2 has twice as many instances and key/value pairs with the same key need to be - // processed by the same - // mapper respectively reducer - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor(); - SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor(); - - ShipStrategyType mapIn = map2Node.getInput().getShipStrategy(); - ShipStrategyType reduceIn = red2Node.getInput().getShipStrategy(); - - Assert.assertTrue( - "Invalid ship strategy for an operator.", - (ShipStrategyType.PARTITION_RANDOM == mapIn - && ShipStrategyType.PARTITION_HASH == reduceIn) - || (ShipStrategyType.PARTITION_HASH == mapIn - && ShipStrategyType.FORWARD == reduceIn)); - } - - @Test - public void checkPropertyHandlingWithDecreasingParallelism() { - final int p = DEFAULT_PARALLELISM; - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(p); - - env.generateSequence(0, 1) - .setParallelism(p * 2) - .map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Map1") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p * 2) - .name("Reduce1") - .map(new IdentityMapper()) - .withForwardedFields("*") - .setParallelism(p) - .name("Map2") - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(p) - .name("Reduce2") - .output(new DiscardingOutputFormat()) - .setParallelism(p) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when reducer 1 distributes its data across the instances of map2, it needs to employ a - // local hash method, - // because map2 has twice as many instances and key/value pairs with the same key need to be - // processed by the same - // mapper respectively reducer - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SingleInputPlanNode red2Node = (SingleInputPlanNode) sinkNode.getPredecessor(); - SingleInputPlanNode map2Node = (SingleInputPlanNode) red2Node.getPredecessor(); - - Assert.assertTrue( - "The no sorting local strategy.", - LocalStrategy.SORT == red2Node.getInput().getLocalStrategy() - || LocalStrategy.SORT == map2Node.getInput().getLocalStrategy()); - - Assert.assertTrue( - "The no partitioning ship strategy.", - ShipStrategyType.PARTITION_HASH == red2Node.getInput().getShipStrategy() - || ShipStrategyType.PARTITION_HASH - == map2Node.getInput().getShipStrategy()); - } - - /** - * Checks that re-partitioning happens when the inputs of a two-input contract have different - * parallelisms. - * - *

Test Plan: - * - *

-     *
-     * (source) -> reduce -\
-     *                      Match -> (sink)
-     * (source) -> reduce -/
-     *
-     * 
- */ - @Test - public void checkPropertyHandlingWithTwoInputs() { - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet set1 = env.generateSequence(0, 1).setParallelism(5); - DataSet set2 = env.generateSequence(0, 1).setParallelism(7); - - DataSet reduce1 = - set1.groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(5); - DataSet reduce2 = - set2.groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .withForwardedFields("*") - .setParallelism(7); - - reduce1.join(reduce2) - .where("*") - .equalTo("*") - .with(new IdentityJoiner()) - .setParallelism(5) - .output(new DiscardingOutputFormat()) - .setParallelism(5); - - Plan plan = env.createProgramPlan(); - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - JobGraphGenerator jobGen = new JobGraphGenerator(); - - // Compile plan to verify that no error is thrown - jobGen.compileJobGraph(oPlan); - - oPlan.accept( - new Visitor() { - - @Override - public boolean preVisit(PlanNode visitable) { - if (visitable instanceof DualInputPlanNode) { - DualInputPlanNode node = (DualInputPlanNode) visitable; - Channel c1 = node.getInput1(); - Channel c2 = node.getInput2(); - - Assert.assertEquals( - "Incompatible shipping strategy chosen for match", - ShipStrategyType.FORWARD, - c1.getShipStrategy()); - Assert.assertEquals( - "Incompatible shipping strategy chosen for match", - ShipStrategyType.PARTITION_HASH, - c2.getShipStrategy()); - return false; - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) { - // DO NOTHING - } - }); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java deleted file mode 100644 index 913cc152b720d..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitionPushdownTest.java +++ /dev/null @@ -1,112 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.util.FieldList; -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.Tuple3; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class PartitionPushdownTest extends CompilerTestBase { - - @Test - public void testPartitioningNotPushedDown() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input.groupBy(0, 1) - .sum(2) - .groupBy(0) - .sum(1) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - - SingleInputPlanNode agg2Reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode agg2Combiner = - (SingleInputPlanNode) agg2Reducer.getInput().getSource(); - SingleInputPlanNode agg1Reducer = - (SingleInputPlanNode) agg2Combiner.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_HASH, agg2Reducer.getInput().getShipStrategy()); - assertEquals(new FieldList(0), agg2Reducer.getInput().getShipStrategyKeys()); - - assertEquals(ShipStrategyType.FORWARD, agg2Combiner.getInput().getShipStrategy()); - - assertEquals(ShipStrategyType.PARTITION_HASH, agg1Reducer.getInput().getShipStrategy()); - assertEquals(new FieldList(0, 1), agg1Reducer.getInput().getShipStrategyKeys()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitioningReused() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input.groupBy(0) - .sum(1) - .groupBy(0, 1) - .sum(2) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - - SingleInputPlanNode agg2Reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode agg1Reducer = - (SingleInputPlanNode) agg2Reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, agg2Reducer.getInput().getShipStrategy()); - - assertEquals(ShipStrategyType.PARTITION_HASH, agg1Reducer.getInput().getShipStrategy()); - assertEquals(new FieldList(0), agg1Reducer.getInput().getShipStrategyKeys()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java deleted file mode 100644 index 204522373df90..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java +++ /dev/null @@ -1,1005 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase; -import org.apache.flink.api.common.operators.util.FieldList; -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.Tuple3; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.util.Collector; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -@SuppressWarnings("serial") -public class PartitioningReusageTest extends CompilerTestBase { - - @Test - public void noPreviousPartitioningJoin1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.join(set2, JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(0, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void noPreviousPartitioningJoin2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.join(set2, JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseSinglePartitioningJoin1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .join(set2, JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(0, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseSinglePartitioningJoin2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .join(set2, JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseSinglePartitioningJoin3() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.join( - set2.partitionByHash(2, 1) - .map(new MockMapper()) - .withForwardedFields("2;1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseSinglePartitioningJoin4() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0) - .map(new MockMapper()) - .withForwardedFields("0") - .join(set2, JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseSinglePartitioningJoin5() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.join( - set2.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .join( - set2.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(0, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .join( - set2.partitionByHash(1, 2) - .map(new MockMapper()) - .withForwardedFields("1;2"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin3() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0) - .map(new MockMapper()) - .withForwardedFields("0") - .join( - set2.partitionByHash(2, 1) - .map(new MockMapper()) - .withForwardedFields("2;1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 1) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin4() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0, 2) - .map(new MockMapper()) - .withForwardedFields("0;2") - .join( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 2) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin5() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2") - .join( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 2) - .equalTo(2, 1) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin6() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(0) - .map(new MockMapper()) - .withForwardedFields("0") - .join( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 2) - .equalTo(1, 2) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void reuseBothPartitioningJoin7() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> joined = - set1.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2") - .join( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1"), - JoinOperatorBase.JoinHint.REPARTITION_HASH_FIRST) - .where(0, 2) - .equalTo(1, 2) - .with(new MockJoin()); - - joined.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidJoinInputProperties(join); - } - - @Test - public void noPreviousPartitioningCoGroup1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.coGroup(set2).where(0, 1).equalTo(0, 1).with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void noPreviousPartitioningCoGroup2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.coGroup(set2).where(0, 1).equalTo(2, 1).with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseSinglePartitioningCoGroup1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .coGroup(set2) - .where(0, 1) - .equalTo(0, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseSinglePartitioningCoGroup2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .coGroup(set2) - .where(0, 1) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseSinglePartitioningCoGroup3() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.coGroup( - set2.partitionByHash(2, 1) - .map(new MockMapper()) - .withForwardedFields("2;1")) - .where(0, 1) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseSinglePartitioningCoGroup4() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0) - .map(new MockMapper()) - .withForwardedFields("0") - .coGroup(set2) - .where(0, 1) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseSinglePartitioningCoGroup5() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.coGroup(set2.partitionByHash(2).map(new MockMapper()).withForwardedFields("2")) - .where(0, 1) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup1() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .coGroup( - set2.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1")) - .where(0, 1) - .equalTo(0, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup2() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0, 1) - .map(new MockMapper()) - .withForwardedFields("0;1") - .coGroup( - set2.partitionByHash(1, 2) - .map(new MockMapper()) - .withForwardedFields("1;2")) - .where(0, 1) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup3() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0) - .map(new MockMapper()) - .withForwardedFields("0") - .coGroup( - set2.partitionByHash(2, 1) - .map(new MockMapper()) - .withForwardedFields("2;1")) - .where(0, 1) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup4() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(0, 2) - .map(new MockMapper()) - .withForwardedFields("0;2") - .coGroup( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1")) - .where(0, 2) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup5() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2") - .coGroup( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1")) - .where(0, 2) - .equalTo(2, 1) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup6() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2") - .coGroup( - set2.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2")) - .where(0, 2) - .equalTo(1, 2) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - @Test - public void reuseBothPartitioningCoGroup7() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - DataSet> coGrouped = - set1.partitionByHash(2) - .map(new MockMapper()) - .withForwardedFields("2") - .coGroup( - set2.partitionByHash(1) - .map(new MockMapper()) - .withForwardedFields("1")) - .where(0, 2) - .equalTo(1, 2) - .with(new MockCoGroup()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - checkValidCoGroupInputProperties(coGroup); - } - - private void checkValidJoinInputProperties(DualInputPlanNode join) { - - GlobalProperties inProps1 = join.getInput1().getGlobalProperties(); - GlobalProperties inProps2 = join.getInput2().getGlobalProperties(); - - if (inProps1.getPartitioning() == PartitioningProperty.HASH_PARTITIONED - && inProps2.getPartitioning() == PartitioningProperty.HASH_PARTITIONED) { - - // check that both inputs are hash partitioned on the same fields - FieldList pFields1 = inProps1.getPartitioningFields(); - FieldList pFields2 = inProps2.getPartitioningFields(); - - assertTrue( - "Inputs are not the same number of fields. Input 1: " - + pFields1 - + ", Input 2: " - + pFields2, - pFields1.size() == pFields2.size()); - - FieldList reqPFields1 = join.getKeysForInput1(); - FieldList reqPFields2 = join.getKeysForInput2(); - - for (int i = 0; i < pFields1.size(); i++) { - - // get fields - int f1 = pFields1.get(i); - int f2 = pFields2.get(i); - - // check that field positions in original key field list are identical - int pos1 = getPosInFieldList(f1, reqPFields1); - int pos2 = getPosInFieldList(f2, reqPFields2); - - if (pos1 < 0) { - fail( - "Input 1 is partitioned on field " - + f1 - + " which is not contained in the key set " - + reqPFields1); - } - if (pos2 < 0) { - fail( - "Input 2 is partitioned on field " - + f2 - + " which is not contained in the key set " - + reqPFields2); - } - if (pos1 != pos2) { - fail("Inputs are not partitioned on the same key fields"); - } - } - - } else if (inProps1.getPartitioning() == PartitioningProperty.FULL_REPLICATION - && inProps2.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED) { - // we are good. No need to check for fields - } else if (inProps1.getPartitioning() == PartitioningProperty.RANDOM_PARTITIONED - && inProps2.getPartitioning() == PartitioningProperty.FULL_REPLICATION) { - // we are good. No need to check for fields - } else { - throw new UnsupportedOperationException( - "This method has only been implemented to check for hash partitioned coGroupinputs"); - } - } - - private void checkValidCoGroupInputProperties(DualInputPlanNode coGroup) { - - GlobalProperties inProps1 = coGroup.getInput1().getGlobalProperties(); - GlobalProperties inProps2 = coGroup.getInput2().getGlobalProperties(); - - if (inProps1.getPartitioning() == PartitioningProperty.HASH_PARTITIONED - && inProps2.getPartitioning() == PartitioningProperty.HASH_PARTITIONED) { - - // check that both inputs are hash partitioned on the same fields - FieldList pFields1 = inProps1.getPartitioningFields(); - FieldList pFields2 = inProps2.getPartitioningFields(); - - assertTrue( - "Inputs are not the same number of fields. Input 1: " - + pFields1 - + ", Input 2: " - + pFields2, - pFields1.size() == pFields2.size()); - - FieldList reqPFields1 = coGroup.getKeysForInput1(); - FieldList reqPFields2 = coGroup.getKeysForInput2(); - - for (int i = 0; i < pFields1.size(); i++) { - - // get fields - int f1 = pFields1.get(i); - int f2 = pFields2.get(i); - - // check that field positions in original key field list are identical - int pos1 = getPosInFieldList(f1, reqPFields1); - int pos2 = getPosInFieldList(f2, reqPFields2); - - if (pos1 < 0) { - fail( - "Input 1 is partitioned on field " - + f1 - + " which is not contained in the key set " - + reqPFields1); - } - if (pos2 < 0) { - fail( - "Input 2 is partitioned on field " - + f2 - + " which is not contained in the key set " - + reqPFields2); - } - if (pos1 != pos2) { - fail("Inputs are not partitioned on the same key fields"); - } - } - - } else { - throw new UnsupportedOperationException( - "This method has only been implemented to check for hash partitioned coGroup inputs"); - } - } - - private int getPosInFieldList(int field, FieldList list) { - - int pos; - for (pos = 0; pos < list.size(); pos++) { - if (field == list.get(pos)) { - break; - } - } - if (pos == list.size()) { - return -1; - } else { - return pos; - } - } - - public static class MockMapper - implements MapFunction< - Tuple3, Tuple3> { - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - return null; - } - } - - public static class MockJoin - implements JoinFunction< - Tuple3, - Tuple3, - Tuple3> { - - @Override - public Tuple3 join( - Tuple3 first, Tuple3 second) - throws Exception { - return null; - } - } - - public static class MockCoGroup - implements CoGroupFunction< - Tuple3, - Tuple3, - Tuple3> { - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception {} - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java deleted file mode 100644 index e559fcf1e6a38..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PipelineBreakerTest.java +++ /dev/null @@ -1,309 +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.optimizer; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.Plan; -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.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; - -import org.junit.Test; - -import static org.junit.Assert.*; - -/** - * Tests in this class validate that the {@link ExecutionMode#PIPELINED} execution mode properly - * sets batch data exchanges, to guard against deadlocks, but does not place pipeline breakers. - */ -@SuppressWarnings("serial") -public class PipelineBreakerTest extends CompilerTestBase { - - @Test - public void testPipelineBreakerWithBroadcastVariable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(ExecutionMode.PIPELINED); - env.setParallelism(64); - - DataSet source = env.generateSequence(1, 10).map(new IdentityMapper()); - - DataSet result = - source.map(new IdentityMapper()) - .map(new IdentityMapper()) - .withBroadcastSet(source, "bc"); - - result.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode mapperInput = (SingleInputPlanNode) mapper.getInput().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, mapper.getBroadcastInputs().get(0).getTempMode()); - - assertEquals(DataExchangeMode.BATCH, mapperInput.getInput().getDataExchangeMode()); - assertEquals( - DataExchangeMode.BATCH, - mapper.getBroadcastInputs().get(0).getDataExchangeMode()); - - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPipelineBreakerBroadcastedAllReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(ExecutionMode.PIPELINED); - env.setParallelism(64); - - DataSet sourceWithMapper = - env.generateSequence(1, 10).map(new IdentityMapper()); - - DataSet bcInput1 = - sourceWithMapper - .map(new IdentityMapper()) - .reduce(new SelectOneReducer()); - DataSet bcInput2 = env.generateSequence(1, 10); - - DataSet result = - sourceWithMapper - .map(new IdentityMapper()) - .withBroadcastSet(bcInput1, "bc1") - .withBroadcastSet(bcInput2, "bc2"); - - result.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(DataExchangeMode.BATCH, mapper.getInput().getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * - * - *
-     *                                +----------- ITERATION ---------+
-     *                                |                               |
-     *                               +--+                           +----+
-     *  (source 1) ----------------->|PS| ------------ +        +-->|next|---> (sink)
-     *                               +--+              | (BC)   |   +----+
-     *                                |                V        |     |
-     *  (source 2) --> (map) --+------|-----------> (MAPPER) ---+     |
-     *                         |      |                ^              |
-     *                         |      |                | (BC)         |
-     *                         |      +----------------|--------------+
-     *                         |                       |
-     *                         +--(map) --> (reduce) --+
-     * 
- */ - @Test - public void testPipelineBreakerBroadcastedPartialSolution() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(ExecutionMode.PIPELINED); - env.setParallelism(64); - - DataSet initialSource = env.generateSequence(1, 10); - IterativeDataSet iteration = initialSource.iterate(100); - - DataSet sourceWithMapper = - env.generateSequence(1, 10).map(new IdentityMapper()); - - DataSet bcInput1 = - sourceWithMapper - .map(new IdentityMapper()) - .reduce(new SelectOneReducer()); - - DataSet result = - sourceWithMapper - .map(new IdentityMapper()) - .withBroadcastSet(iteration, "bc2") - .withBroadcastSet(bcInput1, "bc1"); - - iteration.closeWith(result).output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - BulkIterationPlanNode iterationPlanNode = - (BulkIterationPlanNode) sink.getInput().getSource(); - SingleInputPlanNode mapper = - (SingleInputPlanNode) iterationPlanNode.getRootOfStepFunction(); - - assertEquals(TempMode.CACHED, mapper.getInput().getTempMode()); - assertEquals(DataExchangeMode.BATCH, mapper.getInput().getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPipelineBreakerWithCross() { - try { - { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(64); - - DataSet initialSource = env.generateSequence(1, 10); - - Configuration conf = new Configuration(); - conf.setString( - Optimizer.HINT_LOCAL_STRATEGY, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST); - initialSource - .map(new IdentityMapper()) - .cross(initialSource) - .withParameters(conf) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode cross = (DualInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode mapper = (SingleInputPlanNode) cross.getInput1().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, cross.getInput2().getTempMode()); - assertEquals(DataExchangeMode.BATCH, mapper.getInput().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, cross.getInput2().getDataExchangeMode()); - } - - { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(64); - - DataSet initialSource = env.generateSequence(1, 10); - - Configuration conf = new Configuration(); - conf.setString( - Optimizer.HINT_LOCAL_STRATEGY, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND); - initialSource - .map(new IdentityMapper()) - .cross(initialSource) - .withParameters(conf) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode cross = (DualInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode mapper = (SingleInputPlanNode) cross.getInput1().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, cross.getInput2().getTempMode()); - assertEquals(DataExchangeMode.BATCH, mapper.getInput().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, cross.getInput2().getDataExchangeMode()); - } - - { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(64); - - DataSet initialSource = env.generateSequence(1, 10); - - Configuration conf = new Configuration(); - conf.setString( - Optimizer.HINT_LOCAL_STRATEGY, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST); - initialSource - .map(new IdentityMapper()) - .cross(initialSource) - .withParameters(conf) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode cross = (DualInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode mapper = (SingleInputPlanNode) cross.getInput1().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, cross.getInput2().getTempMode()); - assertEquals(DataExchangeMode.BATCH, mapper.getInput().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, cross.getInput2().getDataExchangeMode()); - } - - { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(64); - - DataSet initialSource = env.generateSequence(1, 10); - - Configuration conf = new Configuration(); - conf.setString( - Optimizer.HINT_LOCAL_STRATEGY, - Optimizer.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND); - initialSource - .map(new IdentityMapper()) - .cross(initialSource) - .withParameters(conf) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode cross = (DualInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode mapper = (SingleInputPlanNode) cross.getInput1().getSource(); - - assertEquals(TempMode.NONE, mapper.getInput().getTempMode()); - assertEquals(TempMode.NONE, cross.getInput2().getTempMode()); - assertEquals(DataExchangeMode.BATCH, mapper.getInput().getDataExchangeMode()); - assertEquals(DataExchangeMode.BATCH, cross.getInput2().getDataExchangeMode()); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java deleted file mode 100644 index 56ac439ff7df4..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PropertyDataSourceTest.java +++ /dev/null @@ -1,919 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -@SuppressWarnings({"serial"}) -public class PropertyDataSourceTest extends CompilerTestBase { - - private List> tuple3PojoData = - new ArrayList>(); - private TupleTypeInfo> tuple3PojoType = - new TupleTypeInfo>( - BasicTypeInfo.LONG_TYPE_INFO, - TypeExtractor.createTypeInfo(SomePojo.class), - BasicTypeInfo.STRING_TYPE_INFO); - - @Test - public void checkSinglePartitionedSource1() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy(0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedSource2() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy(1, 0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(0, 1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedSource3() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("*"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(0, 1, 2, 3, 4))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedSource4() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("f1"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(1, 2, 3))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedSource5() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("f1.stringField"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(3))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedSource6() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("f1.intField; f2"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(2, 4))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedSource7() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy("byDate", 1, 0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(0, 1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING); - Assert.assertTrue(gprops.getCustomPartitioner() != null); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource1() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy(0).splitsGroupedBy(0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(0))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource2() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy(0).splitsGroupedBy(1, 0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(0, 1))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource3() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy(1).splitsGroupedBy(0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource4() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties().splitsPartitionedBy(0, 1).splitsGroupedBy(0); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(0, 1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource5() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("f2").splitsGroupedBy("f2"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(4))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(4))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource6() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties() - .splitsPartitionedBy("f1.intField") - .splitsGroupedBy("f0; f1.intField"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(2))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(0, 2))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource7() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("f1.intField").splitsGroupedBy("f1"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(2))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(1, 2, 3))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedGroupedSource8() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties().splitsPartitionedBy("f1").splitsGroupedBy("f1.stringField"); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(1, 2, 3))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource1() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties() - .splitsPartitionedBy(1) - .splitsOrderedBy(new int[] {1}, new Order[] {Order.ASCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - (new FieldSet(lprops.getGroupedFields().toArray())).equals(new FieldSet(1))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource2() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties() - .splitsPartitionedBy(1) - .splitsOrderedBy(new int[] {1, 0}, new Order[] {Order.ASCENDING, Order.DESCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - (new FieldSet(lprops.getGroupedFields().toArray())).equals(new FieldSet(1, 0))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource3() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties() - .splitsPartitionedBy(0) - .splitsOrderedBy(new int[] {1}, new Order[] {Order.ASCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource4() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data.getSplitDataProperties() - .splitsPartitionedBy(0, 1) - .splitsOrderedBy(new int[] {1}, new Order[] {Order.DESCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(0, 1))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource5() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties() - .splitsPartitionedBy("f1.intField") - .splitsOrderedBy( - "f0; f1.intField", new Order[] {Order.ASCENDING, Order.DESCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(2))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(0, 2))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource6() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties() - .splitsPartitionedBy("f1.intField") - .splitsOrderedBy("f1", new Order[] {Order.DESCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())).equals(new FieldSet(2))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue( - new FieldSet(lprops.getGroupedFields().toArray()).equals(new FieldSet(1, 2, 3))); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkSinglePartitionedOrderedSource7() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data = - env.fromCollection(tuple3PojoData, tuple3PojoType); - - data.getSplitDataProperties() - .splitsPartitionedBy("f1") - .splitsOrderedBy("f1.stringField", new Order[] {Order.ASCENDING}); - - data.output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode = (SourcePlanNode) sinkNode.getPredecessor(); - - GlobalProperties gprops = sourceNode.getGlobalProperties(); - LocalProperties lprops = sourceNode.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops.getPartitioningFields().toArray())) - .equals(new FieldSet(1, 2, 3))); - Assert.assertTrue(gprops.getPartitioning() == PartitioningProperty.ANY_PARTITIONING); - Assert.assertTrue(lprops.getGroupedFields() == null); - Assert.assertTrue(lprops.getOrdering() == null); - } - - @Test - public void checkCoPartitionedSources1() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data1 = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data1.getSplitDataProperties().splitsPartitionedBy("byDate", 0); - - DataSource> data2 = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data2.getSplitDataProperties().splitsPartitionedBy("byDate", 0); - - data1.union(data2).output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode1 = - (SourcePlanNode) - ((NAryUnionPlanNode) sinkNode.getPredecessor()) - .getListOfInputs() - .get(0) - .getSource(); - SourcePlanNode sourceNode2 = - (SourcePlanNode) - ((NAryUnionPlanNode) sinkNode.getPredecessor()) - .getListOfInputs() - .get(1) - .getSource(); - - GlobalProperties gprops1 = sourceNode1.getGlobalProperties(); - LocalProperties lprops1 = sourceNode1.getLocalProperties(); - GlobalProperties gprops2 = sourceNode2.getGlobalProperties(); - LocalProperties lprops2 = sourceNode2.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops1.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops1.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING); - Assert.assertTrue(lprops1.getGroupedFields() == null); - Assert.assertTrue(lprops1.getOrdering() == null); - - Assert.assertTrue( - (new FieldSet(gprops2.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops2.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING); - Assert.assertTrue(lprops2.getGroupedFields() == null); - Assert.assertTrue(lprops2.getOrdering() == null); - - Assert.assertTrue(gprops1.getCustomPartitioner().equals(gprops2.getCustomPartitioner())); - } - - @Test - public void checkCoPartitionedSources2() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSource> data1 = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data1.getSplitDataProperties().splitsPartitionedBy("byCountry", 0); - - DataSource> data2 = - env.readCsvFile("/some/path").types(Long.class, String.class); - - data2.getSplitDataProperties().splitsPartitionedBy("byDate", 0); - - data1.union(data2).output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - SourcePlanNode sourceNode1 = - (SourcePlanNode) - ((NAryUnionPlanNode) sinkNode.getPredecessor()) - .getListOfInputs() - .get(0) - .getSource(); - SourcePlanNode sourceNode2 = - (SourcePlanNode) - ((NAryUnionPlanNode) sinkNode.getPredecessor()) - .getListOfInputs() - .get(1) - .getSource(); - - GlobalProperties gprops1 = sourceNode1.getGlobalProperties(); - LocalProperties lprops1 = sourceNode1.getLocalProperties(); - GlobalProperties gprops2 = sourceNode2.getGlobalProperties(); - LocalProperties lprops2 = sourceNode2.getLocalProperties(); - - Assert.assertTrue( - (new FieldSet(gprops1.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops1.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING); - Assert.assertTrue(lprops1.getGroupedFields() == null); - Assert.assertTrue(lprops1.getOrdering() == null); - - Assert.assertTrue( - (new FieldSet(gprops2.getPartitioningFields().toArray())).equals(new FieldSet(0))); - Assert.assertTrue(gprops2.getPartitioning() == PartitioningProperty.CUSTOM_PARTITIONING); - Assert.assertTrue(lprops2.getGroupedFields() == null); - Assert.assertTrue(lprops2.getOrdering() == null); - - Assert.assertTrue(!gprops1.getCustomPartitioner().equals(gprops2.getCustomPartitioner())); - } - - public static class SomePojo { - public double doubleField; - public int intField; - public String stringField; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java deleted file mode 100644 index 07ab492cb2c6a..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java +++ /dev/null @@ -1,64 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.fail; - -/** - * This test case has been created to validate a bug that occurred when the ReduceOperator was used - * without a grouping key. - */ -@SuppressWarnings({"serial"}) -public class ReduceAllTest extends CompilerTestBase { - - @Test - public void testReduce() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet set1 = env.generateSequence(0, 1); - - set1.reduceGroup(new IdentityGroupReducer()) - .name("Reduce1") - .output(new DiscardingOutputFormat()) - .name("Sink"); - - Plan plan = env.createProgramPlan(); - - try { - OptimizedPlan oPlan = compileNoStats(plan); - JobGraphGenerator jobGen = new JobGraphGenerator(); - jobGen.compileJobGraph(oPlan); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly"); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java deleted file mode 100644 index e71a2184137d1..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java +++ /dev/null @@ -1,554 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.MapPartitionFunction; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.io.ReplicatingInputFormat; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.TupleCsvInputFormat; -import org.apache.flink.api.java.operators.DataSink; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; - -@SuppressWarnings({"serial", "unchecked"}) -public class ReplicatingDataSourceTest extends CompilerTestBase { - - /** Tests join program with replicated data source. */ - @Test - public void checkJoinWithReplicatedSourceInput() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.join(source2).where("*").equalTo("*").writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when join should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy(); - ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2); - } - - /** Tests join program with replicated data source behind map. */ - @Test - public void checkJoinWithReplicatedSourceInputBehindMap() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.map(new IdMap()) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when join should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy(); - ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2); - } - - /** Tests join program with replicated data source behind filter. */ - @Test - public void checkJoinWithReplicatedSourceInputBehindFilter() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.filter(new NoFilter()) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when join should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy(); - ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2); - } - - /** Tests join program with replicated data source behind flatMap. */ - @Test - public void checkJoinWithReplicatedSourceInputBehindFlatMap() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.flatMap(new IdFlatMap()) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when join should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy(); - ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2); - } - - /** Tests join program with replicated data source behind map partition. */ - @Test - public void checkJoinWithReplicatedSourceInputBehindMapPartition() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.mapPartition(new IdPMap()) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when join should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy(); - ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2); - } - - /** Tests join program with replicated data source behind multiple map ops. */ - @Test - public void checkJoinWithReplicatedSourceInputBehindMultiMaps() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.filter(new NoFilter()) - .mapPartition(new IdPMap()) - .flatMap(new IdFlatMap()) - .map(new IdMap()) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when join should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy(); - ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2); - } - - /** Tests cross program with replicated data source. */ - @Test - public void checkCrossWithReplicatedSourceInput() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.cross(source2).writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when cross should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode crossNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType crossIn1 = crossNode.getInput1().getShipStrategy(); - ShipStrategyType crossIn2 = crossNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn2); - } - - /** Tests cross program with replicated data source behind map and filter. */ - @Test - public void checkCrossWithReplicatedSourceInputBehindMap() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.map(new IdMap()) - .filter(new NoFilter()) - .cross(source2) - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - - // check the optimized Plan - // when cross should have forward strategy on both sides - SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode crossNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - ShipStrategyType crossIn1 = crossNode.getInput1().getShipStrategy(); - ShipStrategyType crossIn2 = crossNode.getInput2().getShipStrategy(); - - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn1); - Assert.assertEquals( - "Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn2); - } - - /** - * Tests compiler fail for join program with replicated data source and changing parallelism. - */ - @Test(expected = CompilerException.class) - public void checkJoinWithReplicatedSourceInputChangingparallelism() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.join(source2) - .where("*") - .equalTo("*") - .setParallelism(DEFAULT_PARALLELISM + 2) - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - } - - /** - * Tests compiler fail for join program with replicated data source behind map and changing - * parallelism. - */ - @Test(expected = CompilerException.class) - public void checkJoinWithReplicatedSourceInputBehindMapChangingparallelism() { - - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.map(new IdMap()) - .setParallelism(DEFAULT_PARALLELISM + 1) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - } - - /** Tests compiler fail for join program with replicated data source behind reduce. */ - @Test(expected = CompilerException.class) - public void checkJoinWithReplicatedSourceInputBehindReduce() { - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.reduce(new LastReduce()) - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - } - - /** Tests compiler fail for join program with replicated data source behind rebalance. */ - @Test(expected = CompilerException.class) - public void checkJoinWithReplicatedSourceInputBehindRebalance() { - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - TupleTypeInfo> typeInfo = TupleTypeInfo.getBasicTupleTypeInfo(String.class); - ReplicatingInputFormat, FileInputSplit> rif = - new ReplicatingInputFormat, FileInputSplit>( - new TupleCsvInputFormat>(new Path("/some/path"), typeInfo)); - - DataSet> source1 = - env.createInput( - rif, new TupleTypeInfo>(BasicTypeInfo.STRING_TYPE_INFO)); - DataSet> source2 = env.readCsvFile("/some/otherpath").types(String.class); - - DataSink, Tuple1>> out = - source1.rebalance() - .join(source2) - .where("*") - .equalTo("*") - .writeAsText("/some/newpath"); - - Plan plan = env.createProgramPlan(); - - // submit the plan to the compiler - OptimizedPlan oPlan = compileNoStats(plan); - } - - public static class IdMap implements MapFunction { - - @Override - public T map(T value) throws Exception { - return value; - } - } - - public static class NoFilter implements FilterFunction { - - @Override - public boolean filter(T value) throws Exception { - return false; - } - } - - public static class IdFlatMap implements FlatMapFunction { - - @Override - public void flatMap(T value, Collector out) throws Exception { - out.collect(value); - } - } - - public static class IdPMap implements MapPartitionFunction { - - @Override - public void mapPartition(Iterable values, Collector out) throws Exception { - for (T v : values) { - out.collect(v); - } - } - } - - public static class LastReduce implements ReduceFunction { - - @Override - public T reduce(T value1, T value2) throws Exception { - return value2; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java deleted file mode 100644 index 2499538fe12f9..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java +++ /dev/null @@ -1,213 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; -import org.apache.flink.api.common.operators.base.MapOperatorBase; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -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.Tuple3; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.LocalProperties; -import org.apache.flink.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -import org.junit.Assert; -import org.junit.Test; - -@SuppressWarnings("serial") -public class SemanticPropertiesAPIToPlanTest extends CompilerTestBase { - - @Test - public void forwardFieldsTestMapReduce() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - set = - set.map(new MockMapper()) - .withForwardedFields("*") - .groupBy(0) - .reduce(new MockReducer()) - .withForwardedFields("f0->f1") - .map(new MockMapper()) - .withForwardedFields("*") - .groupBy(1) - .reduce(new MockReducer()) - .withForwardedFields("*"); - - set.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - oPlan.accept( - new Visitor() { - @Override - public boolean preVisit(PlanNode visitable) { - if (visitable instanceof SingleInputPlanNode - && visitable.getProgramOperator() instanceof ReduceOperatorBase) { - for (Channel input : visitable.getInputs()) { - GlobalProperties gprops = visitable.getGlobalProperties(); - LocalProperties lprops = visitable.getLocalProperties(); - - Assert.assertTrue( - "Reduce should just forward the input if it is already partitioned", - input.getShipStrategy() == ShipStrategyType.FORWARD); - Assert.assertTrue( - "Wrong GlobalProperties on Reducer", - gprops.isPartitionedOnFields(new FieldSet(1))); - Assert.assertTrue( - "Wrong GlobalProperties on Reducer", - gprops.getPartitioning() - == PartitioningProperty.HASH_PARTITIONED); - Assert.assertTrue( - "Wrong LocalProperties on Reducer", - lprops.getGroupedFields().contains(1)); - } - } - if (visitable instanceof SingleInputPlanNode - && visitable.getProgramOperator() instanceof MapOperatorBase) { - for (Channel input : visitable.getInputs()) { - GlobalProperties gprops = visitable.getGlobalProperties(); - LocalProperties lprops = visitable.getLocalProperties(); - - Assert.assertTrue( - "Map should just forward the input if it is already partitioned", - input.getShipStrategy() == ShipStrategyType.FORWARD); - Assert.assertTrue( - "Wrong GlobalProperties on Mapper", - gprops.isPartitionedOnFields(new FieldSet(1))); - Assert.assertTrue( - "Wrong GlobalProperties on Mapper", - gprops.getPartitioning() - == PartitioningProperty.HASH_PARTITIONED); - Assert.assertTrue( - "Wrong LocalProperties on Mapper", - lprops.getGroupedFields().contains(1)); - } - return false; - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) {} - }); - } - - @Test - public void forwardFieldsTestJoin() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> in1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> in2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - in1 = - in1.map(new MockMapper()) - .withForwardedFields("*") - .groupBy(0) - .reduce(new MockReducer()) - .withForwardedFields("f0->f1"); - in2 = - in2.map(new MockMapper()) - .withForwardedFields("*") - .groupBy(1) - .reduce(new MockReducer()) - .withForwardedFields("f1->f2"); - DataSet> out = - in1.join(in2).where(1).equalTo(2).with(new MockJoin()); - - out.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - oPlan.accept( - new Visitor() { - @Override - public boolean preVisit(PlanNode visitable) { - if (visitable instanceof DualInputPlanNode - && visitable.getProgramOperator() - instanceof InnerJoinOperatorBase) { - DualInputPlanNode node = ((DualInputPlanNode) visitable); - - final Channel inConn1 = node.getInput1(); - final Channel inConn2 = node.getInput2(); - - Assert.assertTrue( - "Join should just forward the input if it is already partitioned", - inConn1.getShipStrategy() == ShipStrategyType.FORWARD); - Assert.assertTrue( - "Join should just forward the input if it is already partitioned", - inConn2.getShipStrategy() == ShipStrategyType.FORWARD); - return false; - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) {} - }); - } - - public static class MockMapper - implements MapFunction< - Tuple3, Tuple3> { - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - return null; - } - } - - public static class MockReducer implements ReduceFunction> { - @Override - public Tuple3 reduce( - Tuple3 value1, Tuple3 value2) - throws Exception { - return null; - } - } - - public static class MockJoin - implements JoinFunction< - Tuple3, - Tuple3, - Tuple3> { - - @Override - public Tuple3 join( - Tuple3 first, Tuple3 second) - throws Exception { - return null; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java deleted file mode 100644 index de49c85e15396..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java +++ /dev/null @@ -1,133 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -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.Tuple3; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class SortPartialReuseTest extends CompilerTestBase { - - @Test - public void testPartialPartitioningReuse() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input.partitionByHash(0) - .map(new IdentityMapper>()) - .withForwardedFields("0", "1", "2") - .groupBy(0, 1) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .withForwardedFields("0", "1", "2") - .groupBy(0) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer2 = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode reducer1 = (SingleInputPlanNode) reducer2.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - // should be locally forwarding, reusing sort and partitioning - assertEquals(ShipStrategyType.FORWARD, reducer2.getInput().getShipStrategy()); - assertEquals(LocalStrategy.NONE, reducer2.getInput().getLocalStrategy()); - - assertEquals(ShipStrategyType.FORWARD, reducer1.getInput().getShipStrategy()); - assertEquals(LocalStrategy.COMBININGSORT, reducer1.getInput().getLocalStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningNotReused() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input.partitionCustom( - new Partitioner() { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - }, - 0) - .map(new IdentityMapper>()) - .withForwardedFields("0", "1", "2") - .groupBy(0, 1) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .withForwardedFields("0", "1", "2") - .groupBy(1) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer2 = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer2.getInput().getSource(); - SingleInputPlanNode reducer1 = (SingleInputPlanNode) combiner.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - // should be locally forwarding, reusing sort and partitioning - assertEquals(ShipStrategyType.PARTITION_HASH, reducer2.getInput().getShipStrategy()); - assertEquals(LocalStrategy.COMBININGSORT, reducer2.getInput().getLocalStrategy()); - - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy()); - - assertEquals(ShipStrategyType.FORWARD, reducer1.getInput().getShipStrategy()); - assertEquals(LocalStrategy.COMBININGSORT, reducer1.getInput().getLocalStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java deleted file mode 100644 index 0852dfe891cb5..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java +++ /dev/null @@ -1,147 +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.optimizer; - -import org.apache.flink.api.common.Plan; -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.operators.IterativeDataSet; -import org.apache.flink.optimizer.plan.BinaryUnionPlanNode; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class UnionBetweenDynamicAndStaticPathTest extends CompilerTestBase { - - @Test - public void testUnionStaticFirst() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.generateSequence(1, 10); - DataSet input2 = env.generateSequence(1, 10); - - IterativeDataSet iteration = input1.iterate(10); - - DataSet result = - iteration.closeWith(input2.union(input2).union(iteration.union(iteration))); - - result.output(new DiscardingOutputFormat()); - result.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(2, op.getDataSinks().size()); - - BulkIterationPlanNode iterPlan = - (BulkIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - - SingleInputPlanNode noopNode = (SingleInputPlanNode) iterPlan.getRootOfStepFunction(); - BinaryUnionPlanNode mixedUnion = (BinaryUnionPlanNode) noopNode.getInput().getSource(); - NAryUnionPlanNode staticUnion = (NAryUnionPlanNode) mixedUnion.getInput1().getSource(); - NAryUnionPlanNode dynamicUnion = (NAryUnionPlanNode) mixedUnion.getInput2().getSource(); - - assertTrue(mixedUnion.unionsStaticAndDynamicPath()); - assertFalse(mixedUnion.getInput1().isOnDynamicPath()); - assertTrue(mixedUnion.getInput2().isOnDynamicPath()); - assertTrue(mixedUnion.getInput1().getTempMode().isCached()); - - for (Channel c : staticUnion.getInputs()) { - assertFalse(c.isOnDynamicPath()); - } - for (Channel c : dynamicUnion.getInputs()) { - assertTrue(c.isOnDynamicPath()); - } - - assertEquals(0.5, iterPlan.getRelativeMemoryPerSubTask(), 0.0); - assertEquals(0.5, mixedUnion.getInput1().getRelativeTempMemory(), 0.0); - assertEquals(0.0, mixedUnion.getInput2().getRelativeTempMemory(), 0.0); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testUnionStaticSecond() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.generateSequence(1, 10); - DataSet input2 = env.generateSequence(1, 10); - - IterativeDataSet iteration = input1.iterate(10); - - DataSet iterResult = - iteration.closeWith(iteration.union(iteration).union(input2.union(input2))); - - iterResult.output(new DiscardingOutputFormat()); - iterResult.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - assertEquals(2, op.getDataSinks().size()); - - BulkIterationPlanNode iterPlan = - (BulkIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - - SingleInputPlanNode noopNode = (SingleInputPlanNode) iterPlan.getRootOfStepFunction(); - BinaryUnionPlanNode mixedUnion = (BinaryUnionPlanNode) noopNode.getInput().getSource(); - NAryUnionPlanNode staticUnion = (NAryUnionPlanNode) mixedUnion.getInput1().getSource(); - NAryUnionPlanNode dynamicUnion = (NAryUnionPlanNode) mixedUnion.getInput2().getSource(); - - assertTrue(mixedUnion.unionsStaticAndDynamicPath()); - assertFalse(mixedUnion.getInput1().isOnDynamicPath()); - assertTrue(mixedUnion.getInput2().isOnDynamicPath()); - assertTrue(mixedUnion.getInput1().getTempMode().isCached()); - - assertEquals(0.5, iterPlan.getRelativeMemoryPerSubTask(), 0.0); - assertEquals(0.5, mixedUnion.getInput1().getRelativeTempMemory(), 0.0); - assertEquals(0.0, mixedUnion.getInput2().getRelativeTempMemory(), 0.0); - - for (Channel c : staticUnion.getInputs()) { - assertFalse(c.isOnDynamicPath()); - } - for (Channel c : dynamicUnion.getInputs()) { - assertTrue(c.isOnDynamicPath()); - } - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java deleted file mode 100644 index 8bad0b2788661..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java +++ /dev/null @@ -1,193 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.operators.base.FlatMapOperatorBase; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; -import org.apache.flink.util.Visitor; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Iterator; - -@SuppressWarnings({"serial"}) -public class UnionPropertyPropagationTest extends CompilerTestBase { - - @Test - public void testUnion1() { - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet sourceA = env.generateSequence(0, 1); - DataSet sourceB = env.generateSequence(0, 1); - - DataSet redA = sourceA.groupBy("*").reduceGroup(new IdentityGroupReducer()); - DataSet redB = sourceB.groupBy("*").reduceGroup(new IdentityGroupReducer()); - - redA.union(redB) - .groupBy("*") - .reduceGroup(new IdentityGroupReducer()) - .output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - - OptimizedPlan oPlan = compileNoStats(plan); - - JobGraphGenerator jobGen = new JobGraphGenerator(); - - // Compile plan to verify that no error is thrown - jobGen.compileJobGraph(oPlan); - - oPlan.accept( - new Visitor() { - - @Override - public boolean preVisit(PlanNode visitable) { - if (visitable instanceof SingleInputPlanNode - && visitable.getProgramOperator() - instanceof GroupReduceOperatorBase) { - for (Channel inConn : visitable.getInputs()) { - Assert.assertTrue( - "Reduce should just forward the input if it is already partitioned", - inConn.getShipStrategy() == ShipStrategyType.FORWARD); - } - // just check latest ReduceNode - return false; - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) { - // DO NOTHING - } - }); - } - - @Test - public void testUnion2() { - final int NUM_INPUTS = 4; - - // construct the plan it will be multiple flat maps, all unioned - // and the "unioned" inputDataSet will be grouped - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet source = env.readTextFile(IN_FILE); - DataSet> lastUnion = source.flatMap(new DummyFlatMap()); - - for (int i = 1; i < NUM_INPUTS; i++) { - lastUnion = lastUnion.union(source.flatMap(new DummyFlatMap())); - } - - DataSet> result = - lastUnion.groupBy(0).aggregate(Aggregations.SUM, 1); - result.writeAsText(OUT_FILE); - - // return the plan - Plan plan = env.createProgramPlan("Test union on new java-api"); - OptimizedPlan oPlan = compileNoStats(plan); - JobGraphGenerator jobGen = new JobGraphGenerator(); - - // Compile plan to verify that no error is thrown - jobGen.compileJobGraph(oPlan); - - oPlan.accept( - new Visitor() { - - @Override - public boolean preVisit(PlanNode visitable) { - - /* Test on the union output connections - * It must be under the GroupOperator and the strategy should be forward - */ - if (visitable instanceof SingleInputPlanNode - && visitable.getProgramOperator() - instanceof GroupReduceOperatorBase) { - final Channel inConn = ((SingleInputPlanNode) visitable).getInput(); - Assert.assertTrue( - "Union should just forward the Partitioning", - inConn.getShipStrategy() == ShipStrategyType.FORWARD); - Assert.assertTrue( - "Union Node should be under Group operator", - inConn.getSource() instanceof NAryUnionPlanNode); - } - - /* Test on the union input connections - * Must be NUM_INPUTS input connections, all FlatMapOperators with a own partitioning strategy (probably hash) - */ - if (visitable instanceof NAryUnionPlanNode) { - int numberInputs = 0; - for (Iterator inputs = visitable.getInputs().iterator(); - inputs.hasNext(); - numberInputs++) { - final Channel inConn = inputs.next(); - PlanNode inNode = inConn.getSource(); - Assert.assertTrue( - "Input of Union should be FlatMapOperators", - inNode.getProgramOperator() instanceof FlatMapOperatorBase); - Assert.assertTrue( - "Shipment strategy under union should partition the data", - inConn.getShipStrategy() - == ShipStrategyType.PARTITION_HASH); - } - - Assert.assertTrue( - "NAryUnion should have " + NUM_INPUTS + " inputs", - numberInputs == NUM_INPUTS); - return false; - } - return true; - } - - @Override - public void postVisit(PlanNode visitable) { - // DO NOTHING - } - }); - } - - public static final class DummyFlatMap - extends RichFlatMapFunction> { - - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(String value, Collector> out) { - out.collect(new Tuple2(value, 0)); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java deleted file mode 100644 index 1d53fc10f5b61..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java +++ /dev/null @@ -1,657 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.base.JoinOperatorBase; -import org.apache.flink.api.common.operators.util.FieldList; -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.optimizer.dataproperties.PartitioningProperty; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import java.util.List; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class UnionReplacementTest extends CompilerTestBase { - - @Test - public void testUnionReplacement() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet input1 = env.fromElements("test1"); - DataSet input2 = env.fromElements("test2"); - - DataSet union = input1.union(input2); - - union.output(new DiscardingOutputFormat()); - union.output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileNoStats(plan); - JobGraphGenerator jobGen = new JobGraphGenerator(); - jobGen.compileJobGraph(oPlan); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * Test for FLINK-2662. - * - *

Checks that a plan with an union with two outputs is correctly translated. The program can - * be illustrated as follows: - * - *

Src1 ----------------\ >-> Union123 -> GroupBy(0) -> Sum -> Output Src2 -\ / >-> - * Union23--< Src3 -/ \ >-> Union234 -> GroupBy(1) -> Sum -> Output Src4 ----------------/ - * - *

The fix for FLINK-2662 translates the union with two output (Union-23) into two separate - * unions (Union-23_1 and Union-23_2) with one output each. Due to this change, the interesting - * partitioning properties for GroupBy(0) and GroupBy(1) are pushed through Union-23_1 and - * Union-23_2 and do not interfere with each other (which would be the case if Union-23 would be - * a single operator with two outputs). - */ - @Test - public void testUnionWithTwoOutputs() throws Exception { - - // ----------------------------------------------------------------------------------------- - // Build test program - // ----------------------------------------------------------------------------------------- - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> src1 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src2 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src3 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src4 = env.fromElements(new Tuple2<>(0L, 0L)); - - DataSet> union23 = src2.union(src3); - DataSet> union123 = src1.union(union23); - DataSet> union234 = src4.union(union23); - - union123.groupBy(0) - .sum(1) - .name("1") - .output(new DiscardingOutputFormat>()); - union234.groupBy(1) - .sum(0) - .name("2") - .output(new DiscardingOutputFormat>()); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - SingleInputPlanNode groupRed1 = resolver.getNode("1"); - SingleInputPlanNode groupRed2 = resolver.getNode("2"); - - // check partitioning is correct - assertTrue( - "Reduce input should be partitioned on 0.", - groupRed1 - .getInput() - .getGlobalProperties() - .getPartitioningFields() - .isExactMatch(new FieldList(0))); - assertTrue( - "Reduce input should be partitioned on 1.", - groupRed2 - .getInput() - .getGlobalProperties() - .getPartitioningFields() - .isExactMatch(new FieldList(1))); - - // check group reduce inputs are n-ary unions with three inputs - assertTrue( - "Reduce input should be n-ary union with three inputs.", - groupRed1.getInput().getSource() instanceof NAryUnionPlanNode - && ((NAryUnionPlanNode) groupRed1.getInput().getSource()) - .getListOfInputs() - .size() - == 3); - assertTrue( - "Reduce input should be n-ary union with three inputs.", - groupRed2.getInput().getSource() instanceof NAryUnionPlanNode - && ((NAryUnionPlanNode) groupRed2.getInput().getSource()) - .getListOfInputs() - .size() - == 3); - - // check channel from union to group reduce is forwarding - assertTrue( - "Channel between union and group reduce should be forwarding", - groupRed1.getInput().getShipStrategy().equals(ShipStrategyType.FORWARD)); - assertTrue( - "Channel between union and group reduce should be forwarding", - groupRed2.getInput().getShipStrategy().equals(ShipStrategyType.FORWARD)); - - // check that all inputs of unions are hash partitioned - List union123In = - ((NAryUnionPlanNode) groupRed1.getInput().getSource()).getListOfInputs(); - for (Channel i : union123In) { - assertTrue( - "Union input channel should hash partition on 0", - i.getShipStrategy().equals(ShipStrategyType.PARTITION_HASH) - && i.getShipStrategyKeys().isExactMatch(new FieldList(0))); - } - List union234In = - ((NAryUnionPlanNode) groupRed2.getInput().getSource()).getListOfInputs(); - for (Channel i : union234In) { - assertTrue( - "Union input channel should hash partition on 0", - i.getShipStrategy().equals(ShipStrategyType.PARTITION_HASH) - && i.getShipStrategyKeys().isExactMatch(new FieldList(1))); - } - } - - /** - * Checks that a plan with consecutive UNIONs followed by PartitionByHash is correctly - * translated. - * - *

The program can be illustrated as follows: - * - *

Src1 -\ >-> Union12--< Src2 -/ \ >-> Union123 -> PartitionByHash -> Output Src3 - * ----------------/ - * - *

In the resulting plan, the hash partitioning (ShippingStrategy.PARTITION_HASH) must be - * pushed to the inputs of the unions (Src1, Src2, Src3). - */ - @Test - public void testConsecutiveUnionsWithHashPartitioning() throws Exception { - - // ----------------------------------------------------------------------------------------- - // Build test program - // ----------------------------------------------------------------------------------------- - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> src1 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src2 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src3 = env.fromElements(new Tuple2<>(0L, 0L)); - - DataSet> union12 = src1.union(src2); - DataSet> union123 = union12.union(src3); - - union123.partitionByHash(1) - .output(new DiscardingOutputFormat>()) - .name("out"); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - SingleInputPlanNode sink = resolver.getNode("out"); - - // check partitioning is correct - assertEquals( - "Sink input should be hash partitioned.", - PartitioningProperty.HASH_PARTITIONED, - sink.getInput().getGlobalProperties().getPartitioning()); - assertEquals( - "Sink input should be hash partitioned on 1.", - new FieldList(1), - sink.getInput().getGlobalProperties().getPartitioningFields()); - - SingleInputPlanNode partitioner = (SingleInputPlanNode) sink.getInput().getSource(); - assertTrue(partitioner.getDriverStrategy() == DriverStrategy.UNARY_NO_OP); - assertEquals( - "Partitioner input should be hash partitioned.", - PartitioningProperty.HASH_PARTITIONED, - partitioner.getInput().getGlobalProperties().getPartitioning()); - assertEquals( - "Partitioner input should be hash partitioned on 1.", - new FieldList(1), - partitioner.getInput().getGlobalProperties().getPartitioningFields()); - assertEquals( - "Partitioner input channel should be forwarding", - ShipStrategyType.FORWARD, - partitioner.getInput().getShipStrategy()); - - NAryUnionPlanNode union = (NAryUnionPlanNode) partitioner.getInput().getSource(); - // all union inputs should be hash partitioned - for (Channel c : union.getInputs()) { - assertEquals( - "Union input should be hash partitioned", - PartitioningProperty.HASH_PARTITIONED, - c.getGlobalProperties().getPartitioning()); - assertEquals( - "Union input channel should be hash partitioning", - ShipStrategyType.PARTITION_HASH, - c.getShipStrategy()); - assertTrue( - "Union input should be data source", c.getSource() instanceof SourcePlanNode); - } - } - - /** - * Checks that a plan with consecutive UNIONs followed by REBALANCE is correctly translated. - * - *

The program can be illustrated as follows: - * - *

Src1 -\ >-> Union12--< Src2 -/ \ >-> Union123 -> Rebalance -> Output Src3 - * ----------------/ - * - *

In the resulting plan, the Rebalance (ShippingStrategy.PARTITION_FORCED_REBALANCE) must be - * pushed to the inputs of the unions (Src1, Src2, Src3). - */ - @Test - public void testConsecutiveUnionsWithRebalance() throws Exception { - - // ----------------------------------------------------------------------------------------- - // Build test program - // ----------------------------------------------------------------------------------------- - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> src1 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src2 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src3 = env.fromElements(new Tuple2<>(0L, 0L)); - - DataSet> union12 = src1.union(src2); - DataSet> union123 = union12.union(src3); - - union123.rebalance().output(new DiscardingOutputFormat>()).name("out"); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - SingleInputPlanNode sink = resolver.getNode("out"); - - // check partitioning is correct - assertEquals( - "Sink input should be force rebalanced.", - PartitioningProperty.FORCED_REBALANCED, - sink.getInput().getGlobalProperties().getPartitioning()); - - SingleInputPlanNode partitioner = (SingleInputPlanNode) sink.getInput().getSource(); - assertTrue(partitioner.getDriverStrategy() == DriverStrategy.UNARY_NO_OP); - assertEquals( - "Partitioner input should be force rebalanced.", - PartitioningProperty.FORCED_REBALANCED, - partitioner.getInput().getGlobalProperties().getPartitioning()); - assertEquals( - "Partitioner input channel should be forwarding", - ShipStrategyType.FORWARD, - partitioner.getInput().getShipStrategy()); - - NAryUnionPlanNode union = (NAryUnionPlanNode) partitioner.getInput().getSource(); - // all union inputs should be force rebalanced - for (Channel c : union.getInputs()) { - assertEquals( - "Union input should be force rebalanced", - PartitioningProperty.FORCED_REBALANCED, - c.getGlobalProperties().getPartitioning()); - assertEquals( - "Union input channel should be rebalancing", - ShipStrategyType.PARTITION_FORCED_REBALANCE, - c.getShipStrategy()); - assertTrue( - "Union input should be data source", c.getSource() instanceof SourcePlanNode); - } - } - - /** - * Checks that a plan with consecutive UNIONs followed by PARTITION_RANGE is correctly - * translated. - * - *

The program can be illustrated as follows: - * - *

Src1 -\ >-> Union12--< Src2 -/ \ >-> Union123 -> PartitionByRange -> Output Src3 - * ----------------/ - * - *

In the resulting plan, the range partitioning must be pushed to the inputs of the unions - * (Src1, Src2, Src3). - */ - @Test - public void testConsecutiveUnionsWithRangePartitioning() throws Exception { - - // ----------------------------------------------------------------------------------------- - // Build test program - // ----------------------------------------------------------------------------------------- - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> src1 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src2 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src3 = env.fromElements(new Tuple2<>(0L, 0L)); - - DataSet> union12 = src1.union(src2); - DataSet> union123 = union12.union(src3); - - union123.partitionByRange(1) - .output(new DiscardingOutputFormat>()) - .name("out"); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - SingleInputPlanNode sink = resolver.getNode("out"); - - // check partitioning is correct - assertEquals( - "Sink input should be range partitioned.", - PartitioningProperty.RANGE_PARTITIONED, - sink.getInput().getGlobalProperties().getPartitioning()); - assertEquals( - "Sink input should be range partitioned on 1", - new Ordering(1, null, Order.ASCENDING), - sink.getInput().getGlobalProperties().getPartitioningOrdering()); - - SingleInputPlanNode partitioner = (SingleInputPlanNode) sink.getInput().getSource(); - assertTrue(partitioner.getDriverStrategy() == DriverStrategy.UNARY_NO_OP); - assertEquals( - "Partitioner input should be range partitioned.", - PartitioningProperty.RANGE_PARTITIONED, - partitioner.getInput().getGlobalProperties().getPartitioning()); - assertEquals( - "Partitioner input should be range partitioned on 1", - new Ordering(1, null, Order.ASCENDING), - partitioner.getInput().getGlobalProperties().getPartitioningOrdering()); - assertEquals( - "Partitioner input channel should be forwarding", - ShipStrategyType.FORWARD, - partitioner.getInput().getShipStrategy()); - - NAryUnionPlanNode union = (NAryUnionPlanNode) partitioner.getInput().getSource(); - // all union inputs should be range partitioned - for (Channel c : union.getInputs()) { - assertEquals( - "Union input should be range partitioned", - PartitioningProperty.RANGE_PARTITIONED, - c.getGlobalProperties().getPartitioning()); - assertEquals( - "Union input channel should be forwarded", - ShipStrategyType.FORWARD, - c.getShipStrategy()); - // range partitioning is executed as custom partitioning with prior sampling - SingleInputPlanNode partitionMap = (SingleInputPlanNode) c.getSource(); - assertEquals(DriverStrategy.MAP, partitionMap.getDriverStrategy()); - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitionMap.getInput().getShipStrategy()); - } - } - - /** - * Checks that a plan with consecutive UNIONs followed by broadcast-fwd JOIN is correctly - * translated. - * - *

The program can be illustrated as follows: - * - *

Src1 -\ >-> Union12--< Src2 -/ \ >-> Union123 --> bc-fwd-Join -> Output Src3 - * ----------------/ / / Src4 ----------------------------/ - * - *

In the resulting plan, the broadcasting must be pushed to the inputs of the unions (Src1, - * Src2, Src3). - */ - @Test - public void testConsecutiveUnionsWithBroadcast() throws Exception { - - // ----------------------------------------------------------------------------------------- - // Build test program - // ----------------------------------------------------------------------------------------- - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> src1 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src2 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src3 = env.fromElements(new Tuple2<>(0L, 0L)); - DataSet> src4 = env.fromElements(new Tuple2<>(0L, 0L)); - - DataSet> union12 = src1.union(src2); - DataSet> union123 = union12.union(src3); - union123.join(src4, JoinOperatorBase.JoinHint.BROADCAST_HASH_FIRST) - .where(0) - .equalTo(0) - .name("join") - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()) - .name("out"); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - DualInputPlanNode join = resolver.getNode("join"); - - // check input of join is broadcast - assertEquals( - "First join input should be fully replicated.", - PartitioningProperty.FULL_REPLICATION, - join.getInput1().getGlobalProperties().getPartitioning()); - - NAryUnionPlanNode union = (NAryUnionPlanNode) join.getInput1().getSource(); - // check that all union inputs are broadcast - for (Channel c : union.getInputs()) { - assertEquals( - "Union input should be fully replicated", - PartitioningProperty.FULL_REPLICATION, - c.getGlobalProperties().getPartitioning()); - assertEquals( - "Union input channel should be broadcasting", - ShipStrategyType.BROADCAST, - c.getShipStrategy()); - } - } - - /** - * Tests that a the outgoing connection of a Union node is FORWARD. See FLINK-9031 for a bug - * report. - * - *

The issue is quite hard to reproduce as the plan choice seems to depend on the enumeration - * order due to lack of plan costs. This test is a smaller variant of the job that was reported - * to fail. - * - *

/-\ /- PreFilter1 -\-/- Union - PostFilter1 - Reducer1 -\ Src -< >- Union -< X >- Union - - * Out \-/ \- PreFilter2 -/-\- Union - PostFilter2 - Reducer2 -/ - */ - @Test - public void testUnionForwardOutput() throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> src1 = env.fromElements(new Tuple2<>(0L, 0L)); - - DataSet> u1 = src1.union(src1).map(new IdentityMapper<>()); - - DataSet> s1 = u1.filter(x -> true).name("preFilter1"); - DataSet> s2 = u1.filter(x -> true).name("preFilter2"); - - DataSet> reduced1 = - s1.union(s2) - .filter(x -> true) - .name("postFilter1") - .groupBy(0) - .reduceGroup(new IdentityGroupReducer<>()) - .name("reducer1"); - DataSet> reduced2 = - s1.union(s2) - .filter(x -> true) - .name("postFilter2") - .groupBy(1) - .reduceGroup(new IdentityGroupReducer<>()) - .name("reducer2"); - - reduced1.union(reduced2).output(new DiscardingOutputFormat<>()); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - SingleInputPlanNode unionOut1 = resolver.getNode("postFilter1"); - SingleInputPlanNode unionOut2 = resolver.getNode("postFilter2"); - - assertEquals(ShipStrategyType.FORWARD, unionOut1.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, unionOut2.getInput().getShipStrategy()); - } - - /** - * Test the input and output shipping strategies for union operators with input and output - * operators with different parallelisms. - * - *

Src1 - Map(fullP) -\-/- Union - Map(fullP) - Out X Src2 - Map(halfP) -/-\- Union - - * Map(halfP) - Out - * - *

The union operator must always have the same parallelism as its successor and connect to - * it with a FORWARD strategy. In this program, the input connections for union should be - * FORWARD for parallelism-preserving connections and PARTITION_RANDOM for parallelism-changing - * connections. - */ - @Test - public void testUnionInputOutputDifferentDOP() throws Exception { - - int fullDop = DEFAULT_PARALLELISM; - int halfDop = DEFAULT_PARALLELISM / 2; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - DataSet> in1 = - env.fromElements(new Tuple2<>(0L, 0L)) - .map(new IdentityMapper<>()) - .setParallelism(fullDop) - .name("inDopFull"); - DataSet> in2 = - env.fromElements(new Tuple2<>(0L, 0L)) - .map(new IdentityMapper<>()) - .setParallelism(halfDop) - .name("inDopHalf"); - - DataSet> union = in1.union(in2); - - DataSet> dopFullMap = - union.map(new IdentityMapper<>()).setParallelism(fullDop).name("outDopFull"); - DataSet> dopHalfMap = - union.map(new IdentityMapper<>()).setParallelism(halfDop).name("outDopHalf"); - - dopFullMap.output(new DiscardingOutputFormat<>()); - dopHalfMap.output(new DiscardingOutputFormat<>()); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(optimizedPlan); - - SingleInputPlanNode inDopFull = resolver.getNode("inDopFull"); - SingleInputPlanNode inDopHalf = resolver.getNode("inDopHalf"); - SingleInputPlanNode outDopFull = resolver.getNode("outDopFull"); - SingleInputPlanNode outDopHalf = resolver.getNode("outDopHalf"); - NAryUnionPlanNode unionDopFull = (NAryUnionPlanNode) outDopFull.getInput().getSource(); - NAryUnionPlanNode unionDopHalf = (NAryUnionPlanNode) outDopHalf.getInput().getSource(); - - // check in map nodes - assertEquals(2, inDopFull.getOutgoingChannels().size()); - assertEquals(2, inDopHalf.getOutgoingChannels().size()); - assertEquals(fullDop, inDopFull.getParallelism()); - assertEquals(halfDop, inDopHalf.getParallelism()); - - // check union nodes - assertEquals(fullDop, unionDopFull.getParallelism()); - assertEquals(halfDop, unionDopHalf.getParallelism()); - - // check out map nodes - assertEquals(fullDop, outDopFull.getParallelism()); - assertEquals(halfDop, outDopHalf.getParallelism()); - - // check Union -> outMap ship strategies - assertEquals(ShipStrategyType.FORWARD, outDopHalf.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, outDopFull.getInput().getShipStrategy()); - - // check inMap -> Union ship strategies - Channel fullFull; - Channel fullHalf; - Channel halfFull; - Channel halfHalf; - - if (inDopFull.getOutgoingChannels().get(0).getTarget() == unionDopFull) { - fullFull = inDopFull.getOutgoingChannels().get(0); - fullHalf = inDopFull.getOutgoingChannels().get(1); - } else { - fullFull = inDopFull.getOutgoingChannels().get(1); - fullHalf = inDopFull.getOutgoingChannels().get(0); - } - if (inDopHalf.getOutgoingChannels().get(0).getTarget() == unionDopFull) { - halfFull = inDopHalf.getOutgoingChannels().get(0); - halfHalf = inDopHalf.getOutgoingChannels().get(1); - } else { - halfFull = inDopHalf.getOutgoingChannels().get(1); - halfHalf = inDopHalf.getOutgoingChannels().get(0); - } - - assertEquals(ShipStrategyType.FORWARD, fullFull.getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, halfHalf.getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_RANDOM, fullHalf.getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_RANDOM, halfFull.getShipStrategy()); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java deleted file mode 100644 index fe1fffef75e16..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java +++ /dev/null @@ -1,86 +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.optimizer; - -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.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class WorksetIterationCornerCasesTest extends CompilerTestBase { - - @Test - public void testWorksetIterationNotDependingOnSolutionSet() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = - env.generateSequence(1, 100).map(new Duplicator()); - - DeltaIteration, Tuple2> iteration = - input.iterateDelta(input, 100, 1); - - DataSet> iterEnd = - iteration.getWorkset().map(new TestMapper>()); - iteration - .closeWith(iterEnd, iterEnd) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - WorksetIterationPlanNode wipn = - (WorksetIterationPlanNode) - op.getDataSinks().iterator().next().getInput().getSource(); - assertTrue(wipn.getSolutionSetPlanNode().getOutgoingChannels().isEmpty()); - - JobGraphGenerator jgg = new JobGraphGenerator(); - jgg.compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private static final class Duplicator implements MapFunction> { - @Override - public Tuple2 map(T value) { - return new Tuple2(value, value); - } - } - - private static final class TestMapper implements MapFunction { - @Override - public T map(T value) { - return value; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java deleted file mode 100644 index c16dd128574c5..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java +++ /dev/null @@ -1,274 +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.optimizer; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.util.FieldList; -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.operators.DeltaIteration; -import org.apache.flink.api.java.operators.JoinOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer; -import org.apache.flink.optimizer.testfunctions.IdentityJoiner; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Tests that validate optimizer choices when using operators that are requesting certain specific - * execution strategies. - */ -public class WorksetIterationsRecordApiCompilerTest extends CompilerTestBase { - - private static final long serialVersionUID = 1L; - - private static final String ITERATION_NAME = "Test Workset Iteration"; - private static final String JOIN_WITH_INVARIANT_NAME = "Test Join Invariant"; - private static final String JOIN_WITH_SOLUTION_SET = "Test Join SolutionSet"; - private static final String NEXT_WORKSET_REDUCER_NAME = "Test Reduce Workset"; - private static final String SOLUTION_DELTA_MAPPER_NAME = "Test Map Delta"; - - private final FieldList list0 = new FieldList(0); - - @Test - public void testRecordApiWithDeferredSoltionSetUpdateWithMapper() { - Plan plan = getTestPlan(false, true); - - OptimizedPlan oPlan; - try { - oPlan = compileNoStats(plan); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly."); - return; // silence the compiler - } - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); - DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); - SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); - SingleInputPlanNode deltaMapper = resolver.getNode(SOLUTION_DELTA_MAPPER_NAME); - - // iteration preserves partitioning in reducer, so the first partitioning is out of the - // loop, - // the in-loop partitioning is before the final reducer - - // verify joinWithInvariant - assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithInvariantNode.getInput2().getShipStrategy()); - assertEquals(list0, joinWithInvariantNode.getKeysForInput1()); - assertEquals(list0, joinWithInvariantNode.getKeysForInput2()); - - // verify joinWithSolutionSet - assertEquals( - ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); - - // verify reducer - assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); - assertEquals(list0, worksetReducer.getKeys(0)); - - // currently, the system may partition before or after the mapper - ShipStrategyType ss1 = deltaMapper.getInput().getShipStrategy(); - ShipStrategyType ss2 = deltaMapper.getOutgoingChannels().get(0).getShipStrategy(); - - assertTrue( - (ss1 == ShipStrategyType.FORWARD && ss2 == ShipStrategyType.PARTITION_HASH) - || (ss2 == ShipStrategyType.FORWARD - && ss1 == ShipStrategyType.PARTITION_HASH)); - - new JobGraphGenerator().compileJobGraph(oPlan); - } - - @Test - public void testRecordApiWithDeferredSoltionSetUpdateWithNonPreservingJoin() { - Plan plan = getTestPlan(false, false); - - OptimizedPlan oPlan; - try { - oPlan = compileNoStats(plan); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly."); - return; // silence the compiler - } - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); - DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); - SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); - - // iteration preserves partitioning in reducer, so the first partitioning is out of the - // loop, - // the in-loop partitioning is before the final reducer - - // verify joinWithInvariant - assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithInvariantNode.getInput2().getShipStrategy()); - assertEquals(list0, joinWithInvariantNode.getKeysForInput1()); - assertEquals(list0, joinWithInvariantNode.getKeysForInput2()); - - // verify joinWithSolutionSet - assertEquals( - ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); - - // verify reducer - assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); - assertEquals(list0, worksetReducer.getKeys(0)); - - // verify solution delta - assertEquals(2, joinWithSolutionSetNode.getOutgoingChannels().size()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getOutgoingChannels().get(1).getShipStrategy()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } - - @Test - public void testRecordApiWithDirectSoltionSetUpdate() { - Plan plan = getTestPlan(true, false); - - OptimizedPlan oPlan; - try { - oPlan = compileNoStats(plan); - } catch (CompilerException ce) { - ce.printStackTrace(); - fail("The pact compiler is unable to compile this plan correctly."); - return; // silence the compiler - } - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); - DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); - SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); - - // iteration preserves partitioning in reducer, so the first partitioning is out of the - // loop, - // the in-loop partitioning is before the final reducer - - // verify joinWithInvariant - assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithInvariantNode.getInput2().getShipStrategy()); - assertEquals(list0, joinWithInvariantNode.getKeysForInput1()); - assertEquals(list0, joinWithInvariantNode.getKeysForInput2()); - - // verify joinWithSolutionSet - assertEquals( - ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy()); - - // verify reducer - assertEquals(ShipStrategyType.FORWARD, worksetReducer.getInput().getShipStrategy()); - assertEquals(list0, worksetReducer.getKeys(0)); - - // verify solution delta - assertEquals(1, joinWithSolutionSetNode.getOutgoingChannels().size()); - assertEquals( - ShipStrategyType.FORWARD, - joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } - - private Plan getTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) { - - // construct the plan - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - DataSet> solSetInput = - env.readCsvFile("/tmp/sol.csv").types(Long.class, Long.class).name("Solution Set"); - DataSet> workSetInput = - env.readCsvFile("/tmp/sol.csv").types(Long.class, Long.class).name("Workset"); - DataSet> invariantInput = - env.readCsvFile("/tmp/sol.csv") - .types(Long.class, Long.class) - .name("Invariant Input"); - - DeltaIteration, Tuple2> deltaIt = - solSetInput.iterateDelta(workSetInput, 100, 0).name(ITERATION_NAME); - - DataSet> join1 = - deltaIt.getWorkset() - .join(invariantInput) - .where(0) - .equalTo(0) - .with(new IdentityJoiner>()) - .withForwardedFieldsFirst("*") - .name(JOIN_WITH_INVARIANT_NAME); - - DataSet> join2 = - deltaIt.getSolutionSet() - .join(join1) - .where(0) - .equalTo(0) - .with(new IdentityJoiner>()) - .name(JOIN_WITH_SOLUTION_SET); - if (joinPreservesSolutionSet) { - ((JoinOperator) join2).withForwardedFieldsFirst("*"); - } - - DataSet> nextWorkset = - join2.groupBy(0) - .reduceGroup(new IdentityGroupReducer>()) - .withForwardedFields("*") - .name(NEXT_WORKSET_REDUCER_NAME); - - if (mapBeforeSolutionDelta) { - - DataSet> mapper = - join2.map(new IdentityMapper>()) - .withForwardedFields("*") - .name(SOLUTION_DELTA_MAPPER_NAME); - - deltaIt.closeWith(mapper, nextWorkset) - .output(new DiscardingOutputFormat>()); - } else { - deltaIt.closeWith(join2, nextWorkset) - .output(new DiscardingOutputFormat>()); - } - - return env.createProgramPlan(); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java deleted file mode 100644 index 627533befc76a..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java +++ /dev/null @@ -1,443 +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.optimizer.costs; - -import org.apache.flink.optimizer.dag.EstimateProvider; - -import org.junit.Test; - -import static org.junit.Assert.*; - -/** - * Tests for the cost formulas in the {@link DefaultCostEstimator}. Most of the tests establish - * relative relationships. - */ -public class DefaultCostEstimatorTest { - - // estimates - - private static final long SMALL_DATA_SIZE = 10000; - private static final long SMALL_RECORD_COUNT = 100; - - private static final long MEDIUM_DATA_SIZE = 500000000L; - private static final long MEDIUM_RECORD_COUNT = 500000L; - - private static final long BIG_DATA_SIZE = 100000000000L; - private static final long BIG_RECORD_COUNT = 100000000L; - - private static final EstimateProvider UNKNOWN_ESTIMATES = new UnknownEstimates(); - private static final EstimateProvider ZERO_ESTIMATES = new Estimates(0, 0); - private static final EstimateProvider SMALL_ESTIMATES = - new Estimates(SMALL_DATA_SIZE, SMALL_RECORD_COUNT); - private static final EstimateProvider MEDIUM_ESTIMATES = - new Estimates(MEDIUM_DATA_SIZE, MEDIUM_RECORD_COUNT); - private static final EstimateProvider BIG_ESTIMATES = - new Estimates(BIG_DATA_SIZE, BIG_RECORD_COUNT); - - private final CostEstimator costEstimator = new DefaultCostEstimator(); - - // -------------------------------------------------------------------------------------------- - - @Test - public void testShipStrategiesIsolated() { - testShipStrategiesIsolated(UNKNOWN_ESTIMATES, 1); - testShipStrategiesIsolated(UNKNOWN_ESTIMATES, 10); - testShipStrategiesIsolated(ZERO_ESTIMATES, 1); - testShipStrategiesIsolated(ZERO_ESTIMATES, 10); - testShipStrategiesIsolated(SMALL_ESTIMATES, 1); - testShipStrategiesIsolated(SMALL_ESTIMATES, 10); - testShipStrategiesIsolated(BIG_ESTIMATES, 1); - testShipStrategiesIsolated(BIG_ESTIMATES, 10); - } - - private void testShipStrategiesIsolated(EstimateProvider estimates, int targetParallelism) { - Costs random = new Costs(); - costEstimator.addRandomPartitioningCost(estimates, random); - - Costs hash = new Costs(); - costEstimator.addHashPartitioningCost(estimates, hash); - - Costs range = new Costs(); - costEstimator.addRangePartitionCost(estimates, range); - - Costs broadcast = new Costs(); - costEstimator.addBroadcastCost(estimates, targetParallelism, broadcast); - - int randomVsHash = random.compareTo(hash); - int hashVsRange = hash.compareTo(range); - int hashVsBroadcast = hash.compareTo(broadcast); - int rangeVsBroadcast = range.compareTo(broadcast); - - // repartition random is at most as expensive as hash partitioning - assertTrue(randomVsHash <= 0); - - // range partitioning is always more expensive than hash partitioning - assertTrue(hashVsRange < 0); - - // broadcasting is always more expensive than hash partitioning - if (targetParallelism > 1) { - assertTrue(hashVsBroadcast < 0); - } else { - assertTrue(hashVsBroadcast <= 0); - } - - // range partitioning is not more expensive than broadcasting - if (targetParallelism > 1) { - assertTrue(rangeVsBroadcast < 0); - } - } - - // -------------------------------------------------------------------------------------------- - - @Test - public void testShipStrategyCombinationsPlain() { - Costs hashBothSmall = new Costs(); - Costs hashSmallAndLarge = new Costs(); - Costs hashBothLarge = new Costs(); - - Costs hashSmallBcLarge10 = new Costs(); - Costs hashLargeBcSmall10 = new Costs(); - - Costs hashSmallBcLarge1000 = new Costs(); - Costs hashLargeBcSmall1000 = new Costs(); - - Costs forwardSmallBcLarge10 = new Costs(); - Costs forwardLargeBcSmall10 = new Costs(); - - Costs forwardSmallBcLarge1000 = new Costs(); - Costs forwardLargeBcSmall1000 = new Costs(); - - costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashBothSmall); - costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashBothSmall); - - costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashSmallAndLarge); - costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashSmallAndLarge); - - costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashBothLarge); - costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashBothLarge); - - costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashSmallBcLarge10); - costEstimator.addBroadcastCost(BIG_ESTIMATES, 10, hashSmallBcLarge10); - - costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashLargeBcSmall10); - costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 10, hashLargeBcSmall10); - - costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashSmallBcLarge1000); - costEstimator.addBroadcastCost(BIG_ESTIMATES, 1000, hashSmallBcLarge1000); - - costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashLargeBcSmall1000); - costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 1000, hashLargeBcSmall1000); - - costEstimator.addBroadcastCost(BIG_ESTIMATES, 10, forwardSmallBcLarge10); - - costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 10, forwardLargeBcSmall10); - - costEstimator.addBroadcastCost(BIG_ESTIMATES, 1000, forwardSmallBcLarge1000); - - costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 1000, forwardLargeBcSmall1000); - - // hash cost is roughly monotonous - assertTrue(hashBothSmall.compareTo(hashSmallAndLarge) < 0); - assertTrue(hashSmallAndLarge.compareTo(hashBothLarge) < 0); - - // broadcast the smaller is better - assertTrue(hashLargeBcSmall10.compareTo(hashSmallBcLarge10) < 0); - assertTrue(forwardLargeBcSmall10.compareTo(forwardSmallBcLarge10) < 0); - assertTrue(hashLargeBcSmall1000.compareTo(hashSmallBcLarge1000) < 0); - assertTrue(forwardLargeBcSmall1000.compareTo(forwardSmallBcLarge1000) < 0); - - // broadcasting small and forwarding large is better than partition both, given size - // difference - assertTrue(forwardLargeBcSmall10.compareTo(hashSmallAndLarge) < 0); - - // broadcasting too far is expensive again - assertTrue(forwardLargeBcSmall1000.compareTo(hashSmallAndLarge) > 0); - - // assert weight is respected - assertTrue(hashSmallBcLarge10.compareTo(hashSmallBcLarge1000) < 0); - assertTrue(hashLargeBcSmall10.compareTo(hashLargeBcSmall1000) < 0); - assertTrue(forwardSmallBcLarge10.compareTo(forwardSmallBcLarge1000) < 0); - assertTrue(forwardLargeBcSmall10.compareTo(forwardLargeBcSmall1000) < 0); - - // forward versus hash - assertTrue(forwardSmallBcLarge10.compareTo(hashSmallBcLarge10) < 0); - assertTrue(forwardSmallBcLarge1000.compareTo(hashSmallBcLarge1000) < 0); - assertTrue(forwardLargeBcSmall10.compareTo(hashLargeBcSmall10) < 0); - assertTrue(forwardLargeBcSmall1000.compareTo(hashLargeBcSmall1000) < 0); - } - - // -------------------------------------------------------------------------------------------- - - @Test - public void testShipStrategyCombinationsWithUnknowns() { - testShipStrategyCombinationsWithUnknowns(UNKNOWN_ESTIMATES); - testShipStrategyCombinationsWithUnknowns(ZERO_ESTIMATES); - testShipStrategyCombinationsWithUnknowns(SMALL_ESTIMATES); - testShipStrategyCombinationsWithUnknowns(MEDIUM_ESTIMATES); - testShipStrategyCombinationsWithUnknowns(BIG_ESTIMATES); - } - - private void testShipStrategyCombinationsWithUnknowns(EstimateProvider knownEstimates) { - Costs hashBoth = new Costs(); - Costs bcKnown10 = new Costs(); - Costs bcUnknown10 = new Costs(); - Costs bcKnown1000 = new Costs(); - Costs bcUnknown1000 = new Costs(); - - costEstimator.addHashPartitioningCost(knownEstimates, hashBoth); - costEstimator.addHashPartitioningCost(UNKNOWN_ESTIMATES, hashBoth); - - costEstimator.addBroadcastCost(knownEstimates, 10, bcKnown10); - - costEstimator.addBroadcastCost(UNKNOWN_ESTIMATES, 10, bcUnknown10); - - costEstimator.addBroadcastCost(knownEstimates, 1000, bcKnown1000); - - costEstimator.addBroadcastCost(UNKNOWN_ESTIMATES, 1000, bcUnknown1000); - - // if we do not know one of them, hashing both should be cheaper than anything - assertTrue(hashBoth.compareTo(bcKnown10) < 0); - assertTrue(hashBoth.compareTo(bcUnknown10) < 0); - assertTrue(hashBoth.compareTo(bcKnown1000) < 0); - assertTrue(hashBoth.compareTo(bcUnknown1000) < 0); - - // there should be no bias in broadcasting a known or unknown size input - assertTrue(bcKnown10.compareTo(bcUnknown10) == 0); - assertTrue(bcKnown1000.compareTo(bcUnknown1000) == 0); - - // replication factor does matter - assertTrue(bcKnown10.compareTo(bcKnown1000) < 0); - assertTrue(bcUnknown10.compareTo(bcUnknown1000) < 0); - } - - // -------------------------------------------------------------------------------------------- - - @Test - public void testJoinCostFormulasPlain() { - - // hash join costs - - Costs hashBothSmall = new Costs(); - Costs hashBothLarge = new Costs(); - Costs hashSmallBuild = new Costs(); - Costs hashLargeBuild = new Costs(); - - costEstimator.addHybridHashCosts(SMALL_ESTIMATES, BIG_ESTIMATES, hashSmallBuild, 1); - costEstimator.addHybridHashCosts(BIG_ESTIMATES, SMALL_ESTIMATES, hashLargeBuild, 1); - costEstimator.addHybridHashCosts(SMALL_ESTIMATES, SMALL_ESTIMATES, hashBothSmall, 1); - costEstimator.addHybridHashCosts(BIG_ESTIMATES, BIG_ESTIMATES, hashBothLarge, 1); - - assertTrue(hashBothSmall.compareTo(hashSmallBuild) < 0); - assertTrue(hashSmallBuild.compareTo(hashLargeBuild) < 0); - assertTrue(hashLargeBuild.compareTo(hashBothLarge) < 0); - - // merge join costs - - Costs mergeBothSmall = new Costs(); - Costs mergeBothLarge = new Costs(); - Costs mergeSmallFirst = new Costs(); - Costs mergeSmallSecond = new Costs(); - - costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeSmallFirst); - costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeSmallFirst); - costEstimator.addLocalMergeCost(SMALL_ESTIMATES, BIG_ESTIMATES, mergeSmallFirst, 1); - - costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeSmallSecond); - costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeSmallSecond); - costEstimator.addLocalMergeCost(BIG_ESTIMATES, SMALL_ESTIMATES, mergeSmallSecond, 1); - - costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeBothSmall); - costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeBothSmall); - costEstimator.addLocalMergeCost(SMALL_ESTIMATES, SMALL_ESTIMATES, mergeBothSmall, 1); - - costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeBothLarge); - costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeBothLarge); - costEstimator.addLocalMergeCost(BIG_ESTIMATES, BIG_ESTIMATES, mergeBothLarge, 1); - - assertTrue(mergeBothSmall.compareTo(mergeSmallFirst) < 0); - assertTrue(mergeBothSmall.compareTo(mergeSmallSecond) < 0); - assertTrue(mergeSmallFirst.compareTo(mergeSmallSecond) == 0); - assertTrue(mergeSmallFirst.compareTo(mergeBothLarge) < 0); - assertTrue(mergeSmallSecond.compareTo(mergeBothLarge) < 0); - - // compare merge join and hash join costs - - assertTrue(hashBothSmall.compareTo(mergeBothSmall) < 0); - assertTrue(hashBothLarge.compareTo(mergeBothLarge) < 0); - assertTrue(hashSmallBuild.compareTo(mergeSmallFirst) < 0); - assertTrue(hashSmallBuild.compareTo(mergeSmallSecond) < 0); - assertTrue(hashLargeBuild.compareTo(mergeSmallFirst) < 0); - assertTrue(hashLargeBuild.compareTo(mergeSmallSecond) < 0); - } - - // -------------------------------------------------------------------------------------------- - - @Test - public void testJoinCostFormulasWithWeights() { - testJoinCostFormulasWithWeights(UNKNOWN_ESTIMATES, SMALL_ESTIMATES); - testJoinCostFormulasWithWeights(SMALL_ESTIMATES, UNKNOWN_ESTIMATES); - testJoinCostFormulasWithWeights(UNKNOWN_ESTIMATES, MEDIUM_ESTIMATES); - testJoinCostFormulasWithWeights(MEDIUM_ESTIMATES, UNKNOWN_ESTIMATES); - testJoinCostFormulasWithWeights(BIG_ESTIMATES, MEDIUM_ESTIMATES); - testJoinCostFormulasWithWeights(MEDIUM_ESTIMATES, BIG_ESTIMATES); - } - - private void testJoinCostFormulasWithWeights(EstimateProvider e1, EstimateProvider e2) { - Costs hf1 = new Costs(); - Costs hf5 = new Costs(); - Costs hs1 = new Costs(); - Costs hs5 = new Costs(); - Costs mm1 = new Costs(); - Costs mm5 = new Costs(); - - costEstimator.addHybridHashCosts(e1, e2, hf1, 1); - costEstimator.addHybridHashCosts(e1, e2, hf5, 5); - costEstimator.addHybridHashCosts(e2, e1, hs1, 1); - costEstimator.addHybridHashCosts(e2, e1, hs5, 5); - - costEstimator.addLocalSortCost(e1, mm1); - costEstimator.addLocalSortCost(e2, mm1); - costEstimator.addLocalMergeCost(e1, e2, mm1, 1); - - costEstimator.addLocalSortCost(e1, mm5); - costEstimator.addLocalSortCost(e2, mm5); - mm5.multiplyWith(5); - costEstimator.addLocalMergeCost(e1, e2, mm5, 5); - - // weight 1 versus weight 5 - assertTrue(hf1.compareTo(hf5) < 0); - assertTrue(hs1.compareTo(hs5) < 0); - assertTrue(mm1.compareTo(mm5) < 0); - - // hash versus merge - assertTrue(hf1.compareTo(mm1) < 0); - assertTrue(hs1.compareTo(mm1) < 0); - assertTrue(hf5.compareTo(mm5) < 0); - assertTrue(hs5.compareTo(mm5) < 0); - } - - // -------------------------------------------------------------------------------------------- - - @Test - public void testHashJoinCostFormulasWithCaches() { - - Costs hashBothUnknown10 = new Costs(); - Costs hashBothUnknownCached10 = new Costs(); - - Costs hashBothSmall10 = new Costs(); - Costs hashBothSmallCached10 = new Costs(); - - Costs hashSmallLarge10 = new Costs(); - Costs hashSmallLargeCached10 = new Costs(); - - Costs hashLargeSmall10 = new Costs(); - Costs hashLargeSmallCached10 = new Costs(); - - Costs hashLargeSmall1 = new Costs(); - Costs hashLargeSmallCached1 = new Costs(); - - costEstimator.addHybridHashCosts( - UNKNOWN_ESTIMATES, UNKNOWN_ESTIMATES, hashBothUnknown10, 10); - costEstimator.addCachedHybridHashCosts( - UNKNOWN_ESTIMATES, UNKNOWN_ESTIMATES, hashBothUnknownCached10, 10); - - costEstimator.addHybridHashCosts(MEDIUM_ESTIMATES, MEDIUM_ESTIMATES, hashBothSmall10, 10); - costEstimator.addCachedHybridHashCosts( - MEDIUM_ESTIMATES, MEDIUM_ESTIMATES, hashBothSmallCached10, 10); - - costEstimator.addHybridHashCosts(MEDIUM_ESTIMATES, BIG_ESTIMATES, hashSmallLarge10, 10); - costEstimator.addCachedHybridHashCosts( - MEDIUM_ESTIMATES, BIG_ESTIMATES, hashSmallLargeCached10, 10); - - costEstimator.addHybridHashCosts(BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmall10, 10); - costEstimator.addCachedHybridHashCosts( - BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmallCached10, 10); - - costEstimator.addHybridHashCosts(BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmall1, 1); - costEstimator.addCachedHybridHashCosts( - BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmallCached1, 1); - - // cached variant is always cheaper - assertTrue(hashBothUnknown10.compareTo(hashBothUnknownCached10) > 0); - assertTrue(hashBothSmall10.compareTo(hashBothSmallCached10) > 0); - assertTrue(hashSmallLarge10.compareTo(hashSmallLargeCached10) > 0); - assertTrue(hashLargeSmall10.compareTo(hashLargeSmallCached10) > 0); - - // caching the large side is better, because then the small one is the one with additional - // I/O - assertTrue(hashLargeSmallCached10.compareTo(hashSmallLargeCached10) < 0); - - // a weight of one makes the caching the same as the non-cached variant - assertTrue(hashLargeSmall1.compareTo(hashLargeSmallCached1) == 0); - } - - // -------------------------------------------------------------------------------------------- - // Estimate providers - // -------------------------------------------------------------------------------------------- - - private static final class UnknownEstimates implements EstimateProvider { - - @Override - public long getEstimatedOutputSize() { - return -1; - } - - @Override - public long getEstimatedNumRecords() { - return -1; - } - - @Override - public float getEstimatedAvgWidthPerOutputRecord() { - return -1.0f; - } - } - - private static final class Estimates implements EstimateProvider { - - private final long size; - private final long records; - private final float width; - - public Estimates(long size, long records) { - this(size, records, -1.0f); - } - - public Estimates(long size, long records, float width) { - this.size = size; - this.records = records; - this.width = width; - } - - @Override - public long getEstimatedOutputSize() { - return this.size; - } - - @Override - public long getEstimatedNumRecords() { - return this.records; - } - - @Override - public float getEstimatedAvgWidthPerOutputRecord() { - return this.width; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java deleted file mode 100644 index b47e66fe8de5d..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/BinaryCustomPartitioningCompatibilityTest.java +++ /dev/null @@ -1,146 +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.optimizer.custompartition; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -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.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class BinaryCustomPartitioningCompatibilityTest extends CompilerTestBase { - - @Test - public void testCompatiblePartitioningJoin() { - try { - final Partitioner partitioner = - new Partitioner() { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - }; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input1.partitionCustom(partitioner, 1) - .join(input2.partitionCustom(partitioner, 0)) - .where(1) - .equalTo(0) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple3>>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitioner1 = (SingleInputPlanNode) join.getInput1().getSource(); - SingleInputPlanNode partitioner2 = (SingleInputPlanNode) join.getInput2().getSource(); - - assertEquals(ShipStrategyType.FORWARD, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, join.getInput2().getShipStrategy()); - - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner1.getInput().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner2.getInput().getShipStrategy()); - assertEquals(partitioner, partitioner1.getInput().getPartitioner()); - assertEquals(partitioner, partitioner2.getInput().getPartitioner()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCompatiblePartitioningCoGroup() { - try { - final Partitioner partitioner = - new Partitioner() { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - }; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input1.partitionCustom(partitioner, 1) - .coGroup(input2.partitionCustom(partitioner, 0)) - .where(1) - .equalTo(0) - .with(new DummyCoGroupFunction, Tuple3>()) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple3>>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitioner1 = - (SingleInputPlanNode) coGroup.getInput1().getSource(); - SingleInputPlanNode partitioner2 = - (SingleInputPlanNode) coGroup.getInput2().getSource(); - - assertEquals(ShipStrategyType.FORWARD, coGroup.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, coGroup.getInput2().getShipStrategy()); - - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner1.getInput().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner2.getInput().getShipStrategy()); - assertEquals(partitioner, partitioner1.getInput().getPartitioner()); - assertEquals(partitioner, partitioner2.getInput().getPartitioner()); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java deleted file mode 100644 index 170d3e73bcc60..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CoGroupCustomPartitioningTest.java +++ /dev/null @@ -1,316 +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.optimizer.custompartition; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class CoGroupCustomPartitioningTest extends CompilerTestBase { - - @Test - public void testCoGroupWithTuples() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input1.coGroup(input2) - .where(1) - .equalTo(0) - .withPartitioner(partitioner) - .with(new DummyCoGroupFunction, Tuple3>()) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple3>>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCoGroupWithTuplesWrongType() { - try { - final Partitioner partitioner = new TestPartitionerInt(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - try { - input1.coGroup(input2).where(1).equalTo(0).withPartitioner(partitioner); - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCoGroupWithPojos() { - try { - final Partitioner partitioner = new TestPartitionerInt(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - input1.coGroup(input2) - .where("b") - .equalTo("a") - .withPartitioner(partitioner) - .with(new DummyCoGroupFunction()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCoGroupWithPojosWrongType() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - try { - input1.coGroup(input2).where("a").equalTo("b").withPartitioner(partitioner); - - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCoGroupWithKeySelectors() { - try { - final Partitioner partitioner = new TestPartitionerInt(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - input1.coGroup(input2) - .where(new Pojo2KeySelector()) - .equalTo(new Pojo3KeySelector()) - .withPartitioner(partitioner) - .with(new DummyCoGroupFunction()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCoGroupWithKeySelectorsWrongType() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - try { - input1.coGroup(input2) - .where(new Pojo2KeySelector()) - .equalTo(new Pojo3KeySelector()) - .withPartitioner(partitioner); - - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testIncompatibleHashAndCustomPartitioning() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - DataSet> partitioned = - input.partitionCustom( - new Partitioner() { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - }, - 0) - .map(new IdentityMapper>()) - .withForwardedFields("0", "1", "2"); - - DataSet> grouped = - partitioned - .distinct(0, 1) - .groupBy(1) - .sortGroup(0, Order.ASCENDING) - .reduceGroup( - new IdentityGroupReducerCombinable>()) - .withForwardedFields("0", "1"); - - grouped.coGroup(partitioned) - .where(0) - .equalTo(0) - .with( - new DummyCoGroupFunction< - Tuple3, Tuple3>()) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple3>>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_HASH, coGroup.getInput1().getShipStrategy()); - assertTrue( - coGroup.getInput2().getShipStrategy() == ShipStrategyType.PARTITION_HASH - || coGroup.getInput2().getShipStrategy() == ShipStrategyType.FORWARD); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static class TestPartitionerInt implements Partitioner { - @Override - public int partition(Integer key, int numPartitions) { - return 0; - } - } - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } - - public static class Pojo2 { - public int a; - public int b; - } - - public static class Pojo3 { - public int a; - public int b; - public int c; - } - - private static class Pojo2KeySelector implements KeySelector { - @Override - public Integer getKey(Pojo2 value) { - return value.a; - } - } - - private static class Pojo3KeySelector implements KeySelector { - @Override - public Integer getKey(Pojo3 value) { - return value.b; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java deleted file mode 100644 index 5d54797bb024b..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningGlobalOptimizationTest.java +++ /dev/null @@ -1,101 +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.optimizer.custompartition; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -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.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -@SuppressWarnings({"serial", "unchecked"}) -public class CustomPartitioningGlobalOptimizationTest extends CompilerTestBase { - - @Test - public void testJoinReduceCombination() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - DataSet> joined = - input1.join(input2) - .where(1) - .equalTo(0) - .projectFirst(0, 1) - .>projectSecond(2) - .withPartitioner(partitioner); - - joined.groupBy(1) - .withPartitioner(partitioner) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - - assertTrue( - "Reduce is not chained, property reuse does not happen", - reducer.getInput().getSource() instanceof DualInputPlanNode); - - DualInputPlanNode join = (DualInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - - assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java deleted file mode 100644 index 46bfd66a82598..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/CustomPartitioningTest.java +++ /dev/null @@ -1,283 +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.optimizer.custompartition; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityPartitionerMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class CustomPartitioningTest extends CompilerTestBase { - - @Test - public void testPartitionTuples() { - try { - final Partitioner part = new TestPartitionerInt(); - final int parallelism = 4; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)).rebalance(); - - data.partitionCustom(part, 0) - .mapPartition(new IdentityPartitionerMapper>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitioner = (SingleInputPlanNode) mapper.getInput().getSource(); - SingleInputPlanNode balancer = (SingleInputPlanNode) partitioner.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(parallelism, sink.getParallelism()); - - assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - assertEquals(parallelism, mapper.getParallelism()); - - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner.getInput().getShipStrategy()); - assertEquals(part, partitioner.getInput().getPartitioner()); - assertEquals(parallelism, partitioner.getParallelism()); - - assertEquals( - ShipStrategyType.PARTITION_FORCED_REBALANCE, - balancer.getInput().getShipStrategy()); - assertEquals(parallelism, balancer.getParallelism()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitionTuplesInvalidType() { - try { - final int parallelism = 4; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)).rebalance(); - - try { - data.partitionCustom(new TestPartitionerLong(), 0); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitionPojo() { - try { - final Partitioner part = new TestPartitionerInt(); - final int parallelism = 4; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet data = env.fromElements(new Pojo()).rebalance(); - - data.partitionCustom(part, "a") - .mapPartition(new IdentityPartitionerMapper()) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitioner = (SingleInputPlanNode) mapper.getInput().getSource(); - SingleInputPlanNode balancer = (SingleInputPlanNode) partitioner.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(parallelism, sink.getParallelism()); - - assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - assertEquals(parallelism, mapper.getParallelism()); - - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner.getInput().getShipStrategy()); - assertEquals(part, partitioner.getInput().getPartitioner()); - assertEquals(parallelism, partitioner.getParallelism()); - - assertEquals( - ShipStrategyType.PARTITION_FORCED_REBALANCE, - balancer.getInput().getShipStrategy()); - assertEquals(parallelism, balancer.getParallelism()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitionPojoInvalidType() { - try { - final int parallelism = 4; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet data = env.fromElements(new Pojo()).rebalance(); - - try { - data.partitionCustom(new TestPartitionerLong(), "a"); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitionKeySelector() { - try { - final Partitioner part = new TestPartitionerInt(); - final int parallelism = 4; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet data = env.fromElements(new Pojo()).rebalance(); - - data.partitionCustom(part, new TestKeySelectorInt()) - .mapPartition(new IdentityPartitionerMapper()) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode keyRemover = (SingleInputPlanNode) mapper.getInput().getSource(); - SingleInputPlanNode partitioner = - (SingleInputPlanNode) keyRemover.getInput().getSource(); - SingleInputPlanNode keyExtractor = - (SingleInputPlanNode) partitioner.getInput().getSource(); - SingleInputPlanNode balancer = - (SingleInputPlanNode) keyExtractor.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(parallelism, sink.getParallelism()); - - assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - assertEquals(parallelism, mapper.getParallelism()); - - assertEquals(ShipStrategyType.FORWARD, keyRemover.getInput().getShipStrategy()); - assertEquals(parallelism, keyRemover.getParallelism()); - - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner.getInput().getShipStrategy()); - assertEquals(part, partitioner.getInput().getPartitioner()); - assertEquals(parallelism, partitioner.getParallelism()); - - assertEquals(ShipStrategyType.FORWARD, keyExtractor.getInput().getShipStrategy()); - assertEquals(parallelism, keyExtractor.getParallelism()); - - assertEquals( - ShipStrategyType.PARTITION_FORCED_REBALANCE, - balancer.getInput().getShipStrategy()); - assertEquals(parallelism, balancer.getParallelism()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testPartitionKeySelectorInvalidType() { - try { - final Partitioner part = - (Partitioner) (Partitioner) new TestPartitionerLong(); - final int parallelism = 4; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet data = env.fromElements(new Pojo()).rebalance(); - - try { - data.partitionCustom(part, new TestKeySelectorInt()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - public static class Pojo { - public int a; - public int b; - } - - private static class TestPartitionerInt implements Partitioner { - @Override - public int partition(Integer key, int numPartitions) { - return 0; - } - } - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } - - private static class TestKeySelectorInt implements KeySelector { - @Override - public Integer getKey(T value) { - return null; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java deleted file mode 100644 index 0347c0f53e22c..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingKeySelectorTranslationTest.java +++ /dev/null @@ -1,246 +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.optimizer.custompartition; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class GroupingKeySelectorTranslationTest extends CompilerTestBase { - - @Test - public void testCustomPartitioningKeySelectorReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(new TestKeySelector>()) - .withPartitioner(new TestPartitionerInt()) - .reduce(new SelectOneReducer>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode keyRemovingMapper = - (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode reducer = - (SingleInputPlanNode) keyRemovingMapper.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, keyRemovingMapper.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningKeySelectorGroupReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(new TestKeySelector>()) - .withPartitioner(new TestPartitionerInt()) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningKeySelectorGroupReduceSorted() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple3(0, 0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(new TestKeySelector>()) - .withPartitioner(new TestPartitionerInt()) - .sortGroup( - new TestKeySelector>(), - Order.ASCENDING) - .reduceGroup( - new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningKeySelectorInvalidType() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - try { - data.groupBy(new TestKeySelector>()) - .withPartitioner(new TestPartitionerLong()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningKeySelectorInvalidTypeSorted() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple3(0, 0, 0)) - .rebalance() - .setParallelism(4); - - try { - data.groupBy(new TestKeySelector>()) - .sortGroup(1, Order.ASCENDING) - .withPartitioner(new TestPartitionerLong()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleRejectCompositeKey() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple3(0, 0, 0)) - .rebalance() - .setParallelism(4); - - try { - data.groupBy(new TestBinaryKeySelector>()) - .withPartitioner(new TestPartitionerInt()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static class TestPartitionerInt implements Partitioner { - @Override - public int partition(Integer key, int numPartitions) { - return 0; - } - } - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } - - private static class TestKeySelector implements KeySelector { - @Override - public Integer getKey(T value) { - return value.getField(0); - } - } - - private static class TestBinaryKeySelector - implements KeySelector> { - @Override - public Tuple2 getKey(T value) { - return new Tuple2( - value.getField(0), value.getField(1)); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java deleted file mode 100644 index b40e8c4e6d86e..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingPojoTranslationTest.java +++ /dev/null @@ -1,247 +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.optimizer.custompartition; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -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.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class GroupingPojoTranslationTest extends CompilerTestBase { - - @Test - public void testCustomPartitioningTupleReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo2()).rebalance().setParallelism(4); - - data.groupBy("a") - .withPartitioner(new TestPartitionerInt()) - .reduce(new SelectOneReducer()) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleGroupReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo2()).rebalance().setParallelism(4); - - data.groupBy("a") - .withPartitioner(new TestPartitionerInt()) - .reduceGroup(new IdentityGroupReducerCombinable()) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleGroupReduceSorted() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo3()).rebalance().setParallelism(4); - - data.groupBy("a") - .withPartitioner(new TestPartitionerInt()) - .sortGroup("b", Order.ASCENDING) - .reduceGroup(new IdentityGroupReducerCombinable()) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleGroupReduceSorted2() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo4()).rebalance().setParallelism(4); - - data.groupBy("a") - .withPartitioner(new TestPartitionerInt()) - .sortGroup("b", Order.ASCENDING) - .sortGroup("c", Order.DESCENDING) - .reduceGroup(new IdentityGroupReducerCombinable()) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleInvalidType() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo2()).rebalance().setParallelism(4); - - try { - data.groupBy("a").withPartitioner(new TestPartitionerLong()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleInvalidTypeSorted() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo3()).rebalance().setParallelism(4); - - try { - data.groupBy("a") - .sortGroup("b", Order.ASCENDING) - .withPartitioner(new TestPartitionerLong()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleRejectCompositeKey() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(new Pojo2()).rebalance().setParallelism(4); - - try { - data.groupBy("a", "b").withPartitioner(new TestPartitionerInt()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - public static class Pojo2 { - public int a; - public int b; - } - - public static class Pojo3 { - public int a; - public int b; - public int c; - } - - public static class Pojo4 { - public int a; - public int b; - public int c; - public int d; - } - - private static class TestPartitionerInt implements Partitioner { - @Override - public int partition(Integer key, int numPartitions) { - return 0; - } - } - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java deleted file mode 100644 index 77935633e9095..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/GroupingTupleTranslationTest.java +++ /dev/null @@ -1,289 +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.optimizer.custompartition; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -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.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class GroupingTupleTranslationTest extends CompilerTestBase { - - @Test - public void testCustomPartitioningTupleAgg() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(0) - .withPartitioner(new TestPartitionerInt()) - .sum(1) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(0) - .withPartitioner(new TestPartitionerInt()) - .reduce(new SelectOneReducer>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleGroupReduce() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(0) - .withPartitioner(new TestPartitionerInt()) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleGroupReduceSorted() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple3(0, 0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(0) - .withPartitioner(new TestPartitionerInt()) - .sortGroup(1, Order.ASCENDING) - .reduceGroup( - new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleGroupReduceSorted2() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple4(0, 0, 0, 0)) - .rebalance() - .setParallelism(4); - - data.groupBy(0) - .withPartitioner(new TestPartitionerInt()) - .sortGroup(1, Order.ASCENDING) - .sortGroup(2, Order.DESCENDING) - .reduceGroup( - new IdentityGroupReducerCombinable< - Tuple4>()) - .output( - new DiscardingOutputFormat< - Tuple4>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleInvalidType() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple2(0, 0)) - .rebalance() - .setParallelism(4); - - try { - data.groupBy(0).withPartitioner(new TestPartitionerLong()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleInvalidTypeSorted() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple3(0, 0, 0)) - .rebalance() - .setParallelism(4); - - try { - data.groupBy(0) - .sortGroup(1, Order.ASCENDING) - .withPartitioner(new TestPartitionerLong()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningTupleRejectCompositeKey() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(new Tuple3(0, 0, 0)) - .rebalance() - .setParallelism(4); - - try { - data.groupBy(0, 1).withPartitioner(new TestPartitionerInt()); - fail("Should throw an exception"); - } catch (InvalidProgramException e) { - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static class TestPartitionerInt implements Partitioner { - @Override - public int partition(Integer key, int numPartitions) { - return 0; - } - } - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java deleted file mode 100644 index 1e67ab54ae808..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/custompartition/JoinCustomPartitioningTest.java +++ /dev/null @@ -1,317 +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.optimizer.custompartition; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class JoinCustomPartitioningTest extends CompilerTestBase { - - @Test - public void testJoinWithTuples() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - input1.join(input2, JoinHint.REPARTITION_HASH_FIRST) - .where(1) - .equalTo(0) - .withPartitioner(partitioner) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple3>>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testJoinWithTuplesWrongType() { - try { - final Partitioner partitioner = new TestPartitionerInt(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = env.fromElements(new Tuple2(0L, 0L)); - DataSet> input2 = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - try { - input1.join(input2, JoinHint.REPARTITION_HASH_FIRST) - .where(1) - .equalTo(0) - .withPartitioner(partitioner); - - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testJoinWithPojos() { - try { - final Partitioner partitioner = new TestPartitionerInt(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - input1.join(input2, JoinHint.REPARTITION_HASH_FIRST) - .where("b") - .equalTo("a") - .withPartitioner(partitioner) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testJoinWithPojosWrongType() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - try { - input1.join(input2, JoinHint.REPARTITION_HASH_FIRST) - .where("a") - .equalTo("b") - .withPartitioner(partitioner); - - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testJoinWithKeySelectors() { - try { - final Partitioner partitioner = new TestPartitionerInt(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - input1.join(input2, JoinHint.REPARTITION_HASH_FIRST) - .where(new Pojo2KeySelector()) - .equalTo(new Pojo3KeySelector()) - .withPartitioner(partitioner) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_CUSTOM, join.getInput2().getShipStrategy()); - assertEquals(partitioner, join.getInput1().getPartitioner()); - assertEquals(partitioner, join.getInput2().getPartitioner()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testJoinWithKeySelectorsWrongType() { - try { - final Partitioner partitioner = new TestPartitionerLong(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input1 = env.fromElements(new Pojo2()); - DataSet input2 = env.fromElements(new Pojo3()); - - try { - input1.join(input2, JoinHint.REPARTITION_HASH_FIRST) - .where(new Pojo2KeySelector()) - .equalTo(new Pojo3KeySelector()) - .withPartitioner(partitioner); - - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testIncompatibleHashAndCustomPartitioning() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = - env.fromElements(new Tuple3(0L, 0L, 0L)); - - DataSet> partitioned = - input.partitionCustom( - new Partitioner() { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - }, - 0) - .map(new IdentityMapper>()) - .withForwardedFields("0", "1", "2"); - - DataSet> grouped = - partitioned - .distinct(0, 1) - .groupBy(1) - .sortGroup(0, Order.ASCENDING) - .reduceGroup( - new IdentityGroupReducerCombinable>()) - .withForwardedFields("0", "1"); - - grouped.join(partitioned, JoinHint.REPARTITION_HASH_FIRST) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - - assertEquals(ShipStrategyType.PARTITION_HASH, coGroup.getInput1().getShipStrategy()); - assertTrue( - coGroup.getInput2().getShipStrategy() == ShipStrategyType.PARTITION_HASH - || coGroup.getInput2().getShipStrategy() == ShipStrategyType.FORWARD); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static class TestPartitionerInt implements Partitioner { - @Override - public int partition(Integer key, int numPartitions) { - return 0; - } - } - - private static class TestPartitionerLong implements Partitioner { - @Override - public int partition(Long key, int numPartitions) { - return 0; - } - } - - public static class Pojo2 { - public int a; - public int b; - } - - public static class Pojo3 { - public int a; - public int b; - public int c; - } - - private static class Pojo2KeySelector implements KeySelector { - @Override - public Integer getKey(Pojo2 value) { - return value.a; - } - } - - private static class Pojo3KeySelector implements KeySelector { - @Override - public Integer getKey(Pojo3 value) { - return value.b; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/CoGroupNodeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/CoGroupNodeTest.java deleted file mode 100644 index df7bcf0347d96..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/CoGroupNodeTest.java +++ /dev/null @@ -1,96 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.configuration.Configuration; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class CoGroupNodeTest { - - @Test - public void testGetSemanticProperties() { - - DualInputSemanticProperties origProps = new DualInputSemanticProperties(); - // props for first input - origProps.addForwardedField(0, 0, 1); - origProps.addForwardedField(0, 2, 2); - origProps.addForwardedField(0, 3, 4); - origProps.addForwardedField(0, 6, 0); - origProps.addReadFields(0, new FieldSet(0, 2, 4, 7)); - // props for second input - origProps.addForwardedField(1, 1, 2); - origProps.addForwardedField(1, 2, 8); - origProps.addForwardedField(1, 3, 7); - origProps.addForwardedField(1, 6, 6); - origProps.addReadFields(1, new FieldSet(1, 3, 4)); - - CoGroupOperatorBase op = mock(CoGroupOperatorBase.class); - when(op.getSemanticProperties()).thenReturn(origProps); - when(op.getKeyColumns(0)).thenReturn(new int[] {3, 2}); - when(op.getKeyColumns(1)).thenReturn(new int[] {6, 3}); - when(op.getParameters()).thenReturn(new Configuration()); - - CoGroupNode node = new CoGroupNode(op); - - SemanticProperties filteredProps = node.getSemanticPropertiesForLocalPropertyFiltering(); - - // check first input props - assertTrue(filteredProps.getForwardingTargetFields(0, 0).size() == 0); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).contains(2)); - assertTrue(filteredProps.getForwardingTargetFields(0, 3).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(0, 3).contains(4)); - assertTrue(filteredProps.getForwardingTargetFields(0, 6).size() == 0); - assertTrue(filteredProps.getForwardingSourceField(0, 1) < 0); - assertTrue(filteredProps.getForwardingSourceField(0, 2) == 2); - assertTrue(filteredProps.getForwardingSourceField(0, 4) == 3); - assertTrue(filteredProps.getForwardingSourceField(0, 0) < 0); - // check second input props - assertTrue(filteredProps.getReadFields(0).size() == 4); - assertTrue(filteredProps.getReadFields(0).contains(0)); - assertTrue(filteredProps.getReadFields(0).contains(2)); - assertTrue(filteredProps.getReadFields(0).contains(4)); - assertTrue(filteredProps.getReadFields(0).contains(7)); - - assertTrue(filteredProps.getForwardingTargetFields(1, 1).size() == 0); - assertTrue(filteredProps.getForwardingTargetFields(1, 2).size() == 0); - assertTrue(filteredProps.getForwardingTargetFields(1, 3).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(1, 3).contains(7)); - assertTrue(filteredProps.getForwardingTargetFields(1, 6).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(1, 6).contains(6)); - assertTrue(filteredProps.getForwardingSourceField(1, 2) < 0); - assertTrue(filteredProps.getForwardingSourceField(1, 8) < 0); - assertTrue(filteredProps.getForwardingSourceField(1, 7) == 3); - assertTrue(filteredProps.getForwardingSourceField(1, 6) == 6); - - assertTrue(filteredProps.getReadFields(1).size() == 3); - assertTrue(filteredProps.getReadFields(1).contains(1)); - assertTrue(filteredProps.getReadFields(1).contains(3)); - assertTrue(filteredProps.getReadFields(1).contains(4)); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java deleted file mode 100644 index 58c04bbebdaa1..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java +++ /dev/null @@ -1,71 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.configuration.Configuration; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class GroupCombineNodeTest { - - @Test - public void testGetSemanticProperties() { - - SingleInputSemanticProperties origProps = new SingleInputSemanticProperties(); - origProps.addForwardedField(0, 1); - origProps.addForwardedField(2, 2); - origProps.addForwardedField(3, 4); - origProps.addForwardedField(6, 0); - origProps.addReadFields(new FieldSet(0, 2, 4, 7)); - - GroupCombineOperatorBase op = mock(GroupCombineOperatorBase.class); - when(op.getSemanticProperties()).thenReturn(origProps); - when(op.getKeyColumns(0)).thenReturn(new int[] {3, 2}); - when(op.getParameters()).thenReturn(new Configuration()); - - GroupCombineNode node = new GroupCombineNode(op); - - SemanticProperties filteredProps = node.getSemanticPropertiesForLocalPropertyFiltering(); - - assertTrue(filteredProps.getForwardingTargetFields(0, 0).size() == 0); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).contains(2)); - assertTrue(filteredProps.getForwardingTargetFields(0, 3).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(0, 3).contains(4)); - assertTrue(filteredProps.getForwardingTargetFields(0, 6).size() == 0); - assertTrue(filteredProps.getForwardingSourceField(0, 1) < 0); - assertTrue(filteredProps.getForwardingSourceField(0, 2) == 2); - assertTrue(filteredProps.getForwardingSourceField(0, 4) == 3); - assertTrue(filteredProps.getForwardingSourceField(0, 0) < 0); - - assertTrue(filteredProps.getReadFields(0).size() == 4); - assertTrue(filteredProps.getReadFields(0).contains(0)); - assertTrue(filteredProps.getReadFields(0).contains(2)); - assertTrue(filteredProps.getReadFields(0).contains(4)); - assertTrue(filteredProps.getReadFields(0).contains(7)); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupReduceNodeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupReduceNodeTest.java deleted file mode 100644 index be91dbb2741ac..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupReduceNodeTest.java +++ /dev/null @@ -1,71 +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.optimizer.dag; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.configuration.Configuration; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class GroupReduceNodeTest { - - @Test - public void testGetSemanticProperties() { - - SingleInputSemanticProperties origProps = new SingleInputSemanticProperties(); - origProps.addForwardedField(0, 1); - origProps.addForwardedField(2, 2); - origProps.addForwardedField(3, 4); - origProps.addForwardedField(6, 0); - origProps.addReadFields(new FieldSet(0, 2, 4, 7)); - - GroupReduceOperatorBase op = mock(GroupReduceOperatorBase.class); - when(op.getSemanticProperties()).thenReturn(origProps); - when(op.getKeyColumns(0)).thenReturn(new int[] {3, 2}); - when(op.getParameters()).thenReturn(new Configuration()); - - GroupReduceNode node = new GroupReduceNode(op); - - SemanticProperties filteredProps = node.getSemanticPropertiesForLocalPropertyFiltering(); - - assertTrue(filteredProps.getForwardingTargetFields(0, 0).size() == 0); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).contains(2)); - assertTrue(filteredProps.getForwardingTargetFields(0, 3).size() == 1); - assertTrue(filteredProps.getForwardingTargetFields(0, 3).contains(4)); - assertTrue(filteredProps.getForwardingTargetFields(0, 6).size() == 0); - assertTrue(filteredProps.getForwardingSourceField(0, 1) < 0); - assertTrue(filteredProps.getForwardingSourceField(0, 2) == 2); - assertTrue(filteredProps.getForwardingSourceField(0, 4) == 3); - assertTrue(filteredProps.getForwardingSourceField(0, 0) < 0); - - assertTrue(filteredProps.getReadFields(0).size() == 4); - assertTrue(filteredProps.getReadFields(0).contains(0)); - assertTrue(filteredProps.getReadFields(0).contains(2)); - assertTrue(filteredProps.getReadFields(0).contains(4)); - assertTrue(filteredProps.getReadFields(0).contains(7)); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/MapPartitionNodeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/MapPartitionNodeTest.java deleted file mode 100644 index 23ad1c6b69e6d..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/MapPartitionNodeTest.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.optimizer.dag; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.base.MapPartitionOperatorBase; -import org.apache.flink.api.common.operators.util.FieldSet; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class MapPartitionNodeTest { - - @Test - public void testGetSemanticProperties() { - - SingleInputSemanticProperties origProps = new SingleInputSemanticProperties(); - origProps.addForwardedField(0, 1); - origProps.addForwardedField(2, 2); - origProps.addReadFields(new FieldSet(0, 2, 4, 7)); - - MapPartitionOperatorBase op = mock(MapPartitionOperatorBase.class); - when(op.getSemanticProperties()).thenReturn(origProps); - when(op.getKeyColumns(0)).thenReturn(new int[] {}); - - MapPartitionNode node = new MapPartitionNode(op); - - SemanticProperties filteredProps = node.getSemanticPropertiesForLocalPropertyFiltering(); - - assertTrue(filteredProps.getForwardingTargetFields(0, 0).size() == 0); - assertTrue(filteredProps.getForwardingTargetFields(0, 2).size() == 0); - assertTrue(filteredProps.getForwardingSourceField(0, 1) < 0); - assertTrue(filteredProps.getForwardingSourceField(0, 2) < 0); - assertTrue(filteredProps.getReadFields(0).size() == 4); - assertTrue(filteredProps.getReadFields(0).contains(0)); - assertTrue(filteredProps.getReadFields(0).contains(2)); - assertTrue(filteredProps.getReadFields(0).contains(4)); - assertTrue(filteredProps.getReadFields(0).contains(7)); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java deleted file mode 100644 index b05b0be2241da..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeClosedBranchingTest.java +++ /dev/null @@ -1,290 +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.optimizer.dataexchange; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.functions.FilterFunction; -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.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.optimizer.testfunctions.IdentityFlatMapper; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.testfunctions.Top1GroupReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; - -import org.junit.Test; - -import java.util.Collection; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** - * This test checks the correct assignment of the DataExchangeMode to connections for programs that - * branch, and re-join those branches. - * - *

- *                                         /-> (sink)
- *                                        /
- *                         /-> (reduce) -+          /-> (flatmap) -> (sink)
- *                        /               \        /
- *     (source) -> (map) -                (join) -+-----\
- *                        \               /              \
- *                         \-> (filter) -+                \
- *                                       \                (co group) -> (sink)
- *                                        \                /
- *                                         \-> (reduce) - /
- * 
- */ -@SuppressWarnings("serial") -public class DataExchangeModeClosedBranchingTest extends CompilerTestBase { - - @Test - public void testPipelinedForced() { - // PIPELINED_FORCED should result in pipelining all the way - verifyBranchingJoiningPlan( - ExecutionMode.PIPELINED_FORCED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED); - } - - @Test - public void testPipelined() { - // PIPELINED should result in pipelining all the way - verifyBranchingJoiningPlan( - ExecutionMode.PIPELINED, - DataExchangeMode.PIPELINED, // to map - DataExchangeMode.PIPELINED, // to combiner connections are pipelined - DataExchangeMode.BATCH, // to reduce - DataExchangeMode.BATCH, // to filter - DataExchangeMode.PIPELINED, // to sink after reduce - DataExchangeMode.PIPELINED, // to join (first input) - DataExchangeMode.BATCH, // to join (second input) - DataExchangeMode.PIPELINED, // combiner connections are pipelined - DataExchangeMode.BATCH, // to other reducer - DataExchangeMode.PIPELINED, // to flatMap - DataExchangeMode.PIPELINED, // to sink after flatMap - DataExchangeMode.PIPELINED, // to coGroup (first input) - DataExchangeMode.PIPELINED, // to coGroup (second input) - DataExchangeMode.PIPELINED // to sink after coGroup - ); - } - - @Test - public void testBatch() { - // BATCH should result in batching the shuffle all the way - verifyBranchingJoiningPlan( - ExecutionMode.BATCH, - DataExchangeMode.PIPELINED, // to map - DataExchangeMode.PIPELINED, // to combiner connections are pipelined - DataExchangeMode.BATCH, // to reduce - DataExchangeMode.BATCH, // to filter - DataExchangeMode.PIPELINED, // to sink after reduce - DataExchangeMode.BATCH, // to join (first input) - DataExchangeMode.BATCH, // to join (second input) - DataExchangeMode.PIPELINED, // combiner connections are pipelined - DataExchangeMode.BATCH, // to other reducer - DataExchangeMode.PIPELINED, // to flatMap - DataExchangeMode.PIPELINED, // to sink after flatMap - DataExchangeMode.BATCH, // to coGroup (first input) - DataExchangeMode.BATCH, // to coGroup (second input) - DataExchangeMode.PIPELINED // to sink after coGroup - ); - } - - @Test - public void testBatchForced() { - // BATCH_FORCED should result in batching all the way - verifyBranchingJoiningPlan( - ExecutionMode.BATCH_FORCED, - DataExchangeMode.BATCH, // to map - DataExchangeMode.PIPELINED, // to combiner connections are pipelined - DataExchangeMode.BATCH, // to reduce - DataExchangeMode.BATCH, // to filter - DataExchangeMode.BATCH, // to sink after reduce - DataExchangeMode.BATCH, // to join (first input) - DataExchangeMode.BATCH, // to join (second input) - DataExchangeMode.PIPELINED, // combiner connections are pipelined - DataExchangeMode.BATCH, // to other reducer - DataExchangeMode.BATCH, // to flatMap - DataExchangeMode.BATCH, // to sink after flatMap - DataExchangeMode.BATCH, // to coGroup (first input) - DataExchangeMode.BATCH, // to coGroup (second input) - DataExchangeMode.BATCH // to sink after coGroup - ); - } - - private void verifyBranchingJoiningPlan( - ExecutionMode execMode, - DataExchangeMode toMap, - DataExchangeMode toReduceCombiner, - DataExchangeMode toReduce, - DataExchangeMode toFilter, - DataExchangeMode toReduceSink, - DataExchangeMode toJoin1, - DataExchangeMode toJoin2, - DataExchangeMode toOtherReduceCombiner, - DataExchangeMode toOtherReduce, - DataExchangeMode toFlatMap, - DataExchangeMode toFlatMapSink, - DataExchangeMode toCoGroup1, - DataExchangeMode toCoGroup2, - DataExchangeMode toCoGroupSink) { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(execMode); - - // set parallelism to 2 to avoid chaining with source in case when available processors - // is 1. - env.setParallelism(2); - - DataSet> data = - env.fromElements(33L, 44L) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) { - return new Tuple2(value, value); - } - }); - - DataSet> reduced = - data.groupBy(0).reduce(new SelectOneReducer>()); - reduced.output(new DiscardingOutputFormat>()).name("reduceSink"); - - DataSet> filtered = - data.filter( - new FilterFunction>() { - @Override - public boolean filter(Tuple2 value) throws Exception { - return false; - } - }); - - DataSet> joined = - reduced.join(filtered) - .where(1) - .equalTo(1) - .with(new DummyFlatJoinFunction>()); - - joined.flatMap(new IdentityFlatMapper>()) - .output(new DiscardingOutputFormat>()) - .name("flatMapSink"); - - joined.coGroup( - filtered.groupBy(1) - .reduceGroup(new Top1GroupReducer>())) - .where(0) - .equalTo(0) - .with(new DummyCoGroupFunction, Tuple2>()) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()) - .name("cgSink"); - - OptimizedPlan optPlan = compileNoStats(env.createProgramPlan()); - - SinkPlanNode reduceSink = findSink(optPlan.getDataSinks(), "reduceSink"); - SinkPlanNode flatMapSink = findSink(optPlan.getDataSinks(), "flatMapSink"); - SinkPlanNode cgSink = findSink(optPlan.getDataSinks(), "cgSink"); - - DualInputPlanNode coGroupNode = (DualInputPlanNode) cgSink.getPredecessor(); - - DualInputPlanNode joinNode = (DualInputPlanNode) coGroupNode.getInput1().getSource(); - SingleInputPlanNode otherReduceNode = - (SingleInputPlanNode) coGroupNode.getInput2().getSource(); - SingleInputPlanNode otherReduceCombinerNode = - (SingleInputPlanNode) otherReduceNode.getPredecessor(); - - SingleInputPlanNode reduceNode = (SingleInputPlanNode) joinNode.getInput1().getSource(); - SingleInputPlanNode reduceCombinerNode = - (SingleInputPlanNode) reduceNode.getPredecessor(); - assertEquals(reduceNode, reduceSink.getPredecessor()); - - SingleInputPlanNode filterNode = (SingleInputPlanNode) joinNode.getInput2().getSource(); - assertEquals(filterNode, otherReduceCombinerNode.getPredecessor()); - - SingleInputPlanNode mapNode = (SingleInputPlanNode) filterNode.getPredecessor(); - assertEquals(mapNode, reduceCombinerNode.getPredecessor()); - - SingleInputPlanNode flatMapNode = (SingleInputPlanNode) flatMapSink.getPredecessor(); - assertEquals(joinNode, flatMapNode.getPredecessor()); - - // verify the data exchange modes - - assertEquals(toReduceSink, reduceSink.getInput().getDataExchangeMode()); - assertEquals(toFlatMapSink, flatMapSink.getInput().getDataExchangeMode()); - assertEquals(toCoGroupSink, cgSink.getInput().getDataExchangeMode()); - - assertEquals(toCoGroup1, coGroupNode.getInput1().getDataExchangeMode()); - assertEquals(toCoGroup2, coGroupNode.getInput2().getDataExchangeMode()); - - assertEquals(toJoin1, joinNode.getInput1().getDataExchangeMode()); - assertEquals(toJoin2, joinNode.getInput2().getDataExchangeMode()); - - assertEquals(toOtherReduce, otherReduceNode.getInput().getDataExchangeMode()); - assertEquals( - toOtherReduceCombiner, - otherReduceCombinerNode.getInput().getDataExchangeMode()); - - assertEquals(toFlatMap, flatMapNode.getInput().getDataExchangeMode()); - - assertEquals(toFilter, filterNode.getInput().getDataExchangeMode()); - assertEquals(toReduce, reduceNode.getInput().getDataExchangeMode()); - assertEquals(toReduceCombiner, reduceCombinerNode.getInput().getDataExchangeMode()); - - assertEquals(toMap, mapNode.getInput().getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private SinkPlanNode findSink(Collection collection, String name) { - for (SinkPlanNode node : collection) { - String nodeName = node.getOptimizerNode().getOperator().getName(); - if (nodeName != null && nodeName.equals(name)) { - return node; - } - } - - throw new IllegalArgumentException("No node with that name was found."); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java deleted file mode 100644 index 9d1c26f686a0d..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeForwardTest.java +++ /dev/null @@ -1,157 +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.optimizer.dataexchange; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.functions.FilterFunction; -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.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; -import org.apache.flink.optimizer.testfunctions.Top1GroupReducer; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** - * This test verifies that the optimizer assigns the correct data exchange mode to a simple forward - * / shuffle plan. - * - *
- *     (source) -> (map) -> (filter) -> (groupBy / reduce)
- * 
- */ -@SuppressWarnings("serial") -public class DataExchangeModeForwardTest extends CompilerTestBase { - - @Test - public void testPipelinedForced() { - // PIPELINED_FORCED should result in pipelining all the way - verifySimpleForwardPlan( - ExecutionMode.PIPELINED_FORCED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED); - } - - @Test - public void testPipelined() { - // PIPELINED should result in pipelining all the way - verifySimpleForwardPlan( - ExecutionMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED); - } - - @Test - public void testBatch() { - // BATCH should result in batching the shuffle all the way - verifySimpleForwardPlan( - ExecutionMode.BATCH, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.BATCH, - DataExchangeMode.PIPELINED); - } - - @Test - public void testBatchForced() { - // BATCH_FORCED should result in batching all the way - verifySimpleForwardPlan( - ExecutionMode.BATCH_FORCED, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.PIPELINED, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH); - } - - private void verifySimpleForwardPlan( - ExecutionMode execMode, - DataExchangeMode toMap, - DataExchangeMode toFilter, - DataExchangeMode toKeyExtractor, - DataExchangeMode toCombiner, - DataExchangeMode toReduce, - DataExchangeMode toSink) { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(execMode); - - DataSet dataSet = env.readTextFile("/never/accessed"); - dataSet.map( - new MapFunction() { - @Override - public Integer map(String value) { - return 0; - } - }) - .filter( - new FilterFunction() { - @Override - public boolean filter(Integer value) { - return false; - } - }) - .groupBy(new IdentityKeyExtractor()) - .reduceGroup(new Top1GroupReducer()) - .output(new DiscardingOutputFormat()); - - OptimizedPlan optPlan = compileNoStats(env.createProgramPlan()); - SinkPlanNode sinkNode = optPlan.getDataSinks().iterator().next(); - - SingleInputPlanNode reduceNode = (SingleInputPlanNode) sinkNode.getPredecessor(); - SingleInputPlanNode combineNode = (SingleInputPlanNode) reduceNode.getPredecessor(); - SingleInputPlanNode keyExtractorNode = - (SingleInputPlanNode) combineNode.getPredecessor(); - - SingleInputPlanNode filterNode = - (SingleInputPlanNode) keyExtractorNode.getPredecessor(); - SingleInputPlanNode mapNode = (SingleInputPlanNode) filterNode.getPredecessor(); - - assertEquals(toMap, mapNode.getInput().getDataExchangeMode()); - assertEquals(toFilter, filterNode.getInput().getDataExchangeMode()); - assertEquals(toKeyExtractor, keyExtractorNode.getInput().getDataExchangeMode()); - assertEquals(toCombiner, combineNode.getInput().getDataExchangeMode()); - assertEquals(toReduce, reduceNode.getInput().getDataExchangeMode()); - assertEquals(toSink, sinkNode.getInput().getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java deleted file mode 100644 index f9f9c49171390..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/DataExchangeModeOpenBranchingTest.java +++ /dev/null @@ -1,202 +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.optimizer.dataexchange; - -import org.apache.flink.api.common.ExecutionMode; -import org.apache.flink.api.common.functions.FilterFunction; -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.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; - -import org.junit.Test; - -import java.util.Collection; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** - * This test checks the correct assignment of the DataExchangeMode to connections for programs that - * branch, but do not re-join the branches. - * - *
- *                      /---> (filter) -> (sink)
- *                     /
- *                    /
- * (source) -> (map) -----------------\
- *                    \               (join) -> (sink)
- *                     \   (source) --/
- *                      \
- *                       \
- *                        \-> (sink)
- * 
- */ -@SuppressWarnings({"serial", "unchecked"}) -public class DataExchangeModeOpenBranchingTest extends CompilerTestBase { - - @Test - public void testPipelinedForced() { - // PIPELINED_FORCED should result in pipelining all the way - verifyBranchigPlan( - ExecutionMode.PIPELINED_FORCED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED); - } - - @Test - public void testPipelined() { - // PIPELINED should result in pipelining all the way - verifyBranchigPlan( - ExecutionMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED); - } - - @Test - public void testBatch() { - // BATCH should result in batching the shuffle all the way - verifyBranchigPlan( - ExecutionMode.BATCH, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.PIPELINED, - DataExchangeMode.PIPELINED); - } - - @Test - public void testBatchForced() { - // BATCH_FORCED should result in batching all the way - verifyBranchigPlan( - ExecutionMode.BATCH_FORCED, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH, - DataExchangeMode.BATCH); - } - - private void verifyBranchigPlan( - ExecutionMode execMode, - DataExchangeMode toMap, - DataExchangeMode toFilter, - DataExchangeMode toFilterSink, - DataExchangeMode toJoin1, - DataExchangeMode toJoin2, - DataExchangeMode toJoinSink, - DataExchangeMode toDirectSink) { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(execMode); - - DataSet> data = - env.generateSequence(1, 100000) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) { - return new Tuple2(value, value); - } - }); - - // output 1 - data.filter( - new FilterFunction>() { - @Override - public boolean filter(Tuple2 value) { - return false; - } - }) - .output(new DiscardingOutputFormat>()) - .name("sink1"); - - // output 2 does a join before a join - data.join(env.fromElements(new Tuple2(1L, 2L))) - .where(1) - .equalTo(0) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()) - .name("sink2"); - - // output 3 is direct - data.output(new DiscardingOutputFormat>()).name("sink3"); - - OptimizedPlan optPlan = compileNoStats(env.createProgramPlan()); - - SinkPlanNode filterSink = findSink(optPlan.getDataSinks(), "sink1"); - SinkPlanNode joinSink = findSink(optPlan.getDataSinks(), "sink2"); - SinkPlanNode directSink = findSink(optPlan.getDataSinks(), "sink3"); - - SingleInputPlanNode filterNode = (SingleInputPlanNode) filterSink.getPredecessor(); - SingleInputPlanNode mapNode = (SingleInputPlanNode) filterNode.getPredecessor(); - - DualInputPlanNode joinNode = (DualInputPlanNode) joinSink.getPredecessor(); - assertEquals(mapNode, joinNode.getInput1().getSource()); - - assertEquals(mapNode, directSink.getPredecessor()); - - assertEquals(toFilterSink, filterSink.getInput().getDataExchangeMode()); - assertEquals(toJoinSink, joinSink.getInput().getDataExchangeMode()); - assertEquals(toDirectSink, directSink.getInput().getDataExchangeMode()); - - assertEquals(toMap, mapNode.getInput().getDataExchangeMode()); - assertEquals(toFilter, filterNode.getInput().getDataExchangeMode()); - - assertEquals(toJoin1, joinNode.getInput1().getDataExchangeMode()); - assertEquals(toJoin2, joinNode.getInput2().getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private SinkPlanNode findSink(Collection collection, String name) { - for (SinkPlanNode node : collection) { - String nodeName = node.getOptimizerNode().getOperator().getName(); - if (nodeName != null && nodeName.equals(name)) { - return node; - } - } - - throw new IllegalArgumentException("No node with that name was found."); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/PipelineBreakingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/PipelineBreakingTest.java deleted file mode 100644 index 896900319e2fd..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/PipelineBreakingTest.java +++ /dev/null @@ -1,332 +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.optimizer.dataexchange; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.FilterFunction; -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.optimizer.dag.DataSinkNode; -import org.apache.flink.optimizer.dag.OptimizerNode; -import org.apache.flink.optimizer.dag.SingleInputNode; -import org.apache.flink.optimizer.dag.SinkJoiner; -import org.apache.flink.optimizer.dag.TwoInputNode; -import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.optimizer.testfunctions.IdentityFlatMapper; -import org.apache.flink.optimizer.testfunctions.IdentityKeyExtractor; -import org.apache.flink.optimizer.testfunctions.SelectOneReducer; -import org.apache.flink.optimizer.testfunctions.Top1GroupReducer; -import org.apache.flink.optimizer.traversals.BranchesVisitor; -import org.apache.flink.optimizer.traversals.GraphCreatingVisitor; -import org.apache.flink.optimizer.traversals.IdAndEstimatesVisitor; - -import org.junit.Test; - -import java.util.Iterator; -import java.util.List; - -import static org.junit.Assert.*; - -/** This test checks whether connections are correctly marked as pipelined breaking. */ -@SuppressWarnings("serial") -public class PipelineBreakingTest { - - /** - * Tests that no pipeline breakers are inserted into a simple forward pipeline. - * - *
-     *     (source) -> (map) -> (filter) -> (groupBy / reduce)
-     * 
- */ - @Test - public void testSimpleForwardPlan() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet dataSet = env.readTextFile("/never/accessed"); - dataSet.map( - new MapFunction() { - @Override - public Integer map(String value) { - return 0; - } - }) - .filter( - new FilterFunction() { - @Override - public boolean filter(Integer value) { - return false; - } - }) - .groupBy(new IdentityKeyExtractor()) - .reduceGroup(new Top1GroupReducer()) - .output(new DiscardingOutputFormat()); - - DataSinkNode sinkNode = convertPlan(env.createProgramPlan()).get(0); - - SingleInputNode reduceNode = (SingleInputNode) sinkNode.getPredecessorNode(); - SingleInputNode keyExtractorNode = (SingleInputNode) reduceNode.getPredecessorNode(); - - SingleInputNode filterNode = (SingleInputNode) keyExtractorNode.getPredecessorNode(); - SingleInputNode mapNode = (SingleInputNode) filterNode.getPredecessorNode(); - - assertFalse(sinkNode.getInputConnection().isBreakingPipeline()); - assertFalse(reduceNode.getIncomingConnection().isBreakingPipeline()); - assertFalse(keyExtractorNode.getIncomingConnection().isBreakingPipeline()); - assertFalse(filterNode.getIncomingConnection().isBreakingPipeline()); - assertFalse(mapNode.getIncomingConnection().isBreakingPipeline()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * Tests that branching plans, where the branches are not re-joined, do not place pipeline - * breakers. - * - *
-     *                      /---> (filter) -> (sink)
-     *                     /
-     *                    /
-     * (source) -> (map) -----------------\
-     *                    \               (join) -> (sink)
-     *                     \   (source) --/
-     *                      \
-     *                       \
-     *                        \-> (sink)
-     * 
- */ - @Test - public void testBranchingPlanNotReJoined() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = - env.readTextFile("/never/accessed") - .map( - new MapFunction() { - @Override - public Integer map(String value) { - return 0; - } - }); - - // output 1 - data.filter( - new FilterFunction() { - @Override - public boolean filter(Integer value) { - return false; - } - }) - .output(new DiscardingOutputFormat()); - - // output 2 does a join before a join - data.join(env.fromElements(1, 2, 3, 4)) - .where(new IdentityKeyExtractor()) - .equalTo(new IdentityKeyExtractor()) - .output(new DiscardingOutputFormat>()); - - // output 3 is direct - data.output(new DiscardingOutputFormat()); - - List sinks = convertPlan(env.createProgramPlan()); - - // gather the optimizer DAG nodes - - DataSinkNode sinkAfterFilter = sinks.get(0); - DataSinkNode sinkAfterJoin = sinks.get(1); - DataSinkNode sinkDirect = sinks.get(2); - - SingleInputNode filterNode = (SingleInputNode) sinkAfterFilter.getPredecessorNode(); - SingleInputNode mapNode = (SingleInputNode) filterNode.getPredecessorNode(); - - TwoInputNode joinNode = (TwoInputNode) sinkAfterJoin.getPredecessorNode(); - SingleInputNode joinInput = (SingleInputNode) joinNode.getSecondPredecessorNode(); - - // verify the non-pipeline breaking status - - assertFalse(sinkAfterFilter.getInputConnection().isBreakingPipeline()); - assertFalse(sinkAfterJoin.getInputConnection().isBreakingPipeline()); - assertFalse(sinkDirect.getInputConnection().isBreakingPipeline()); - - assertFalse(filterNode.getIncomingConnection().isBreakingPipeline()); - assertFalse(mapNode.getIncomingConnection().isBreakingPipeline()); - - assertFalse(joinNode.getFirstIncomingConnection().isBreakingPipeline()); - assertFalse(joinNode.getSecondIncomingConnection().isBreakingPipeline()); - assertFalse(joinInput.getIncomingConnection().isBreakingPipeline()); - - // some other sanity checks on the plan construction (cannot hurt) - - assertEquals( - mapNode, - ((SingleInputNode) joinNode.getFirstPredecessorNode()).getPredecessorNode()); - assertEquals(mapNode, sinkDirect.getPredecessorNode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * Tests that branches that are re-joined have place pipeline breakers. - * - *
-     *                                         /-> (sink)
-     *                                        /
-     *                         /-> (reduce) -+          /-> (flatmap) -> (sink)
-     *                        /               \        /
-     *     (source) -> (map) -                (join) -+-----\
-     *                        \               /              \
-     *                         \-> (filter) -+                \
-     *                                       \                (co group) -> (sink)
-     *                                        \                /
-     *                                         \-> (reduce) - /
-     * 
- */ - @Test - public void testReJoinedBranches() { - try { - // build a test program - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromElements(33L, 44L) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) { - return new Tuple2(value, value); - } - }); - - DataSet> reduced = - data.groupBy(0).reduce(new SelectOneReducer>()); - reduced.output(new DiscardingOutputFormat>()); - - DataSet> filtered = - data.filter( - new FilterFunction>() { - @Override - public boolean filter(Tuple2 value) throws Exception { - return false; - } - }); - - DataSet> joined = - reduced.join(filtered) - .where(1) - .equalTo(1) - .with(new DummyFlatJoinFunction>()); - - joined.flatMap(new IdentityFlatMapper>()) - .output(new DiscardingOutputFormat>()); - - joined.coGroup( - filtered.groupBy(1) - .reduceGroup(new Top1GroupReducer>())) - .where(0) - .equalTo(0) - .with(new DummyCoGroupFunction, Tuple2>()) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()); - - List sinks = convertPlan(env.createProgramPlan()); - - // gather the optimizer DAG nodes - - DataSinkNode sinkAfterReduce = sinks.get(0); - DataSinkNode sinkAfterFlatMap = sinks.get(1); - DataSinkNode sinkAfterCoGroup = sinks.get(2); - - SingleInputNode reduceNode = (SingleInputNode) sinkAfterReduce.getPredecessorNode(); - SingleInputNode mapNode = (SingleInputNode) reduceNode.getPredecessorNode(); - - SingleInputNode flatMapNode = (SingleInputNode) sinkAfterFlatMap.getPredecessorNode(); - TwoInputNode joinNode = (TwoInputNode) flatMapNode.getPredecessorNode(); - SingleInputNode filterNode = (SingleInputNode) joinNode.getSecondPredecessorNode(); - - TwoInputNode coGroupNode = (TwoInputNode) sinkAfterCoGroup.getPredecessorNode(); - SingleInputNode otherReduceNode = - (SingleInputNode) coGroupNode.getSecondPredecessorNode(); - - // test sanity checks (that we constructed the DAG correctly) - - assertEquals(reduceNode, joinNode.getFirstPredecessorNode()); - assertEquals(mapNode, filterNode.getPredecessorNode()); - assertEquals(joinNode, coGroupNode.getFirstPredecessorNode()); - assertEquals(filterNode, otherReduceNode.getPredecessorNode()); - - // verify the pipeline breaking status - - assertFalse(sinkAfterReduce.getInputConnection().isBreakingPipeline()); - assertFalse(sinkAfterFlatMap.getInputConnection().isBreakingPipeline()); - assertFalse(sinkAfterCoGroup.getInputConnection().isBreakingPipeline()); - - assertFalse(mapNode.getIncomingConnection().isBreakingPipeline()); - assertFalse(flatMapNode.getIncomingConnection().isBreakingPipeline()); - assertFalse(joinNode.getFirstIncomingConnection().isBreakingPipeline()); - assertFalse(coGroupNode.getFirstIncomingConnection().isBreakingPipeline()); - assertFalse(coGroupNode.getSecondIncomingConnection().isBreakingPipeline()); - - // these should be pipeline breakers - assertTrue(reduceNode.getIncomingConnection().isBreakingPipeline()); - assertTrue(filterNode.getIncomingConnection().isBreakingPipeline()); - assertTrue(otherReduceNode.getIncomingConnection().isBreakingPipeline()); - assertTrue(joinNode.getSecondIncomingConnection().isBreakingPipeline()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private static List convertPlan(Plan p) { - GraphCreatingVisitor dagCreator = - new GraphCreatingVisitor(17, p.getExecutionConfig().getExecutionMode()); - - // create the DAG - p.accept(dagCreator); - List sinks = dagCreator.getSinks(); - - // build a single root and run the branch tracking logic - OptimizerNode rootNode; - if (sinks.size() == 1) { - rootNode = sinks.get(0); - } else { - Iterator iter = sinks.iterator(); - rootNode = iter.next(); - - while (iter.hasNext()) { - rootNode = new SinkJoiner(rootNode, iter.next()); - } - } - rootNode.accept(new IdAndEstimatesVisitor(null)); - rootNode.accept(new BranchesVisitor()); - - return sinks; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/UnionClosedBranchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/UnionClosedBranchingTest.java deleted file mode 100644 index 6e824ff4f429d..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataexchange/UnionClosedBranchingTest.java +++ /dev/null @@ -1,215 +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.optimizer.dataexchange; - -import org.apache.flink.api.common.ExecutionMode; -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.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSet; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.runtime.io.network.DataExchangeMode.BATCH; -import static org.apache.flink.runtime.io.network.DataExchangeMode.PIPELINED; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -/** - * This tests a fix for FLINK-2540. - * - *

This test is necessary, because {@link NAryUnionPlanNode}s are not directly translated to - * runtime tasks by the {@link JobGraphGenerator}. Instead, the network stack unions the inputs by - * directly reading from multiple inputs (via {@link UnionInputGate}). - * - *

- *   (source)-\        /-\
- *            (union)-+  (join)
- *   (source)-/        \-/
- * 
- * - * @see FLINK-2540 - */ -@RunWith(Parameterized.class) -@SuppressWarnings({"serial", "unchecked"}) -public class UnionClosedBranchingTest extends CompilerTestBase { - - @Parameterized.Parameters - public static Collection params() { - Collection params = - Arrays.asList( - new Object[][] { - {ExecutionMode.PIPELINED, BATCH, PIPELINED}, - {ExecutionMode.PIPELINED_FORCED, PIPELINED, PIPELINED}, - {ExecutionMode.BATCH, BATCH, PIPELINED}, - {ExecutionMode.BATCH_FORCED, BATCH, BATCH}, - }); - - // Make sure that changes to ExecutionMode are reflected in this test. - assertEquals(ExecutionMode.values().length, params.size()); - - return params; - } - - private final ExecutionMode executionMode; - - /** Expected {@link DataExchangeMode} from sources to union. */ - private final DataExchangeMode sourceToUnion; - - /** Expected {@link DataExchangeMode} from union to join. */ - private final DataExchangeMode unionToJoin; - - /** Expected {@link ShipStrategyType} from source to union. */ - private final ShipStrategyType sourceToUnionStrategy = ShipStrategyType.PARTITION_HASH; - - /** Expected {@link ShipStrategyType} from union to join. */ - private final ShipStrategyType unionToJoinStrategy = ShipStrategyType.FORWARD; - - public UnionClosedBranchingTest( - ExecutionMode executionMode, - DataExchangeMode sourceToUnion, - DataExchangeMode unionToJoin) { - - this.executionMode = executionMode; - this.sourceToUnion = sourceToUnion; - this.unionToJoin = unionToJoin; - } - - @Test - public void testUnionClosedBranchingTest() throws Exception { - - // ----------------------------------------------------------------------------------------- - // Build test program - // ----------------------------------------------------------------------------------------- - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().setExecutionMode(executionMode); - env.setParallelism(4); - - DataSet> src1 = env.fromElements(new Tuple1<>(0), new Tuple1<>(1)); - - DataSet> src2 = env.fromElements(new Tuple1<>(0), new Tuple1<>(1)); - - DataSet> union = src1.union(src2); - - DataSet> join = - union.join(union).where(0).equalTo(0).projectFirst(0).projectSecond(0); - - join.output(new DiscardingOutputFormat>()); - - // ----------------------------------------------------------------------------------------- - // Verify optimized plan - // ----------------------------------------------------------------------------------------- - - OptimizedPlan optimizedPlan = compileNoStats(env.createProgramPlan()); - - SinkPlanNode sinkNode = optimizedPlan.getDataSinks().iterator().next(); - - DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor(); - - // Verify that the compiler correctly sets the expected data exchange modes. - for (Channel channel : joinNode.getInputs()) { - assertEquals( - "Unexpected data exchange mode between union and join node.", - unionToJoin, - channel.getDataExchangeMode()); - assertEquals( - "Unexpected ship strategy between union and join node.", - unionToJoinStrategy, - channel.getShipStrategy()); - } - - for (SourcePlanNode src : optimizedPlan.getDataSources()) { - for (Channel channel : src.getOutgoingChannels()) { - assertEquals( - "Unexpected data exchange mode between source and union node.", - sourceToUnion, - channel.getDataExchangeMode()); - assertEquals( - "Unexpected ship strategy between source and union node.", - sourceToUnionStrategy, - channel.getShipStrategy()); - } - } - - // ----------------------------------------------------------------------------------------- - // Verify generated JobGraph - // ----------------------------------------------------------------------------------------- - - JobGraphGenerator jgg = new JobGraphGenerator(); - JobGraph jobGraph = jgg.compileJobGraph(optimizedPlan); - - List vertices = jobGraph.getVerticesSortedTopologicallyFromSources(); - - // Sanity check for the test setup - assertEquals("Unexpected number of vertices created.", 4, vertices.size()); - - // Verify all sources - JobVertex[] sources = new JobVertex[] {vertices.get(0), vertices.get(1)}; - - for (JobVertex src : sources) { - // Sanity check - assertTrue("Unexpected vertex type. Test setup is broken.", src.isInputVertex()); - - // The union is not translated to an extra union task, but the join uses a union - // input gate to read multiple inputs. The source create a single result per consumer. - assertEquals( - "Unexpected number of created results.", - 2, - src.getNumberOfProducedIntermediateDataSets()); - - for (IntermediateDataSet dataSet : src.getProducedDataSets()) { - ResultPartitionType dsType = dataSet.getResultType(); - - // Ensure batch exchange unless PIPELINED_FORCE is enabled. - if (!executionMode.equals(ExecutionMode.PIPELINED_FORCED)) { - assertTrue( - "Expected batch exchange, but result type is " + dsType + ".", - dsType.isBlockingOrBlockingPersistentResultPartition()); - } else { - assertFalse( - "Expected non-batch exchange, but result type is " + dsType + ".", - dsType.isBlockingOrBlockingPersistentResultPartition()); - } - } - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesFilteringTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesFilteringTest.java deleted file mode 100644 index 912cd0d03177f..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesFilteringTest.java +++ /dev/null @@ -1,462 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Set; - -import static org.junit.Assert.*; - -public class GlobalPropertiesFilteringTest { - - private TupleTypeInfo< - Tuple8> - tupleInfo = - new TupleTypeInfo< - Tuple8< - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer>>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - public void testAllErased1() { - - SingleInputSemanticProperties semProps = new SingleInputSemanticProperties(); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setHashPartitioned(new FieldList(0, 1)); - gprops.addUniqueFieldCombination(new FieldSet(3, 4)); - gprops.addUniqueFieldCombination(new FieldSet(5, 6)); - - GlobalProperties result = gprops.filterBySemanticProperties(semProps, 0); - - assertEquals(PartitioningProperty.RANDOM_PARTITIONED, result.getPartitioning()); - assertNull(result.getPartitioningFields()); - assertNull(result.getPartitioningOrdering()); - assertNull(result.getUniqueFieldCombination()); - } - - @Test - public void testAllErased2() { - - SingleInputSemanticProperties semProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - semProps, new String[] {"2"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setHashPartitioned(new FieldList(0, 1)); - gprops.addUniqueFieldCombination(new FieldSet(3, 4)); - gprops.addUniqueFieldCombination(new FieldSet(5, 6)); - - GlobalProperties result = gprops.filterBySemanticProperties(semProps, 0); - - assertEquals(PartitioningProperty.RANDOM_PARTITIONED, result.getPartitioning()); - assertNull(result.getPartitioningFields()); - assertNull(result.getPartitioningOrdering()); - assertNull(result.getUniqueFieldCombination()); - } - - @Test - public void testHashPartitioningPreserved1() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1;4"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setHashPartitioned(new FieldList(0, 1, 4)); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.HASH_PARTITIONED, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(3, pFields.size()); - assertTrue(pFields.contains(0)); - assertTrue(pFields.contains(1)); - assertTrue(pFields.contains(4)); - } - - @Test - public void testHashPartitioningPreserved2() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0->1; 1->2; 4->3"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setHashPartitioned(new FieldList(0, 1, 4)); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.HASH_PARTITIONED, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(3, pFields.size()); - assertTrue(pFields.contains(1)); - assertTrue(pFields.contains(2)); - assertTrue(pFields.contains(3)); - } - - @Test - public void testHashPartitioningErased() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setHashPartitioned(new FieldList(0, 1, 4)); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.RANDOM_PARTITIONED, result.getPartitioning()); - assertNull(result.getPartitioningFields()); - } - - @Test - public void testAnyPartitioningPreserved1() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1;4"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setAnyPartitioning(new FieldList(0, 1, 4)); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.ANY_PARTITIONING, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(3, pFields.size()); - assertTrue(pFields.contains(0)); - assertTrue(pFields.contains(1)); - assertTrue(pFields.contains(4)); - } - - @Test - public void testAnyPartitioningPreserved2() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0->1; 1->2; 4->3"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setAnyPartitioning(new FieldList(0, 1, 4)); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.ANY_PARTITIONING, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(3, pFields.size()); - assertTrue(pFields.contains(1)); - assertTrue(pFields.contains(2)); - assertTrue(pFields.contains(3)); - } - - @Test - public void testAnyPartitioningErased() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setAnyPartitioning(new FieldList(0, 1, 4)); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.RANDOM_PARTITIONED, result.getPartitioning()); - assertNull(result.getPartitioningFields()); - } - - @Test - public void testCustomPartitioningPreserved1() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1;4"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - Partitioner> myP = new MockPartitioner(); - gprops.setCustomPartitioned(new FieldList(0, 4), myP); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.CUSTOM_PARTITIONING, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(2, pFields.size()); - assertTrue(pFields.contains(0)); - assertTrue(pFields.contains(4)); - assertEquals(myP, result.getCustomPartitioner()); - } - - @Test - public void testCustomPartitioningPreserved2() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0->1; 1->2; 4->3"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - Partitioner> myP = new MockPartitioner(); - gprops.setCustomPartitioned(new FieldList(0, 4), myP); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.CUSTOM_PARTITIONING, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(2, pFields.size()); - assertTrue(pFields.contains(1)); - assertTrue(pFields.contains(3)); - assertEquals(myP, result.getCustomPartitioner()); - } - - @Test - public void testCustomPartitioningErased() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - Partitioner> myP = new MockPartitioner(); - gprops.setCustomPartitioned(new FieldList(0, 4), myP); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.RANDOM_PARTITIONED, result.getPartitioning()); - assertNull(result.getPartitioningFields()); - assertNull(result.getCustomPartitioner()); - } - - @Test - public void testRangePartitioningPreserved1() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"1;2;5"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - o.appendOrdering(2, StringValue.class, Order.ASCENDING); - GlobalProperties gprops = new GlobalProperties(); - gprops.setRangePartitioned(o); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.RANGE_PARTITIONED, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(3, pFields.size()); - assertEquals(1, pFields.get(0).intValue()); - assertEquals(5, pFields.get(1).intValue()); - assertEquals(2, pFields.get(2).intValue()); - Ordering pOrder = result.getPartitioningOrdering(); - assertEquals(3, pOrder.getNumberOfFields()); - assertEquals(1, pOrder.getFieldNumber(0).intValue()); - assertEquals(5, pOrder.getFieldNumber(1).intValue()); - assertEquals(2, pOrder.getFieldNumber(2).intValue()); - assertEquals(Order.ASCENDING, pOrder.getOrder(0)); - assertEquals(Order.DESCENDING, pOrder.getOrder(1)); - assertEquals(Order.ASCENDING, pOrder.getOrder(2)); - assertEquals(IntValue.class, pOrder.getType(0)); - assertEquals(LongValue.class, pOrder.getType(1)); - assertEquals(StringValue.class, pOrder.getType(2)); - } - - @Test - public void testRangePartitioningPreserved2() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"1->3; 2->0; 5->1"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - o.appendOrdering(2, StringValue.class, Order.ASCENDING); - GlobalProperties gprops = new GlobalProperties(); - gprops.setRangePartitioned(o); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.RANGE_PARTITIONED, result.getPartitioning()); - FieldList pFields = result.getPartitioningFields(); - assertEquals(3, pFields.size()); - assertEquals(3, pFields.get(0).intValue()); - assertEquals(1, pFields.get(1).intValue()); - assertEquals(0, pFields.get(2).intValue()); - Ordering pOrder = result.getPartitioningOrdering(); - assertEquals(3, pOrder.getNumberOfFields()); - assertEquals(3, pOrder.getFieldNumber(0).intValue()); - assertEquals(1, pOrder.getFieldNumber(1).intValue()); - assertEquals(0, pOrder.getFieldNumber(2).intValue()); - assertEquals(Order.ASCENDING, pOrder.getOrder(0)); - assertEquals(Order.DESCENDING, pOrder.getOrder(1)); - assertEquals(Order.ASCENDING, pOrder.getOrder(2)); - assertEquals(IntValue.class, pOrder.getType(0)); - assertEquals(LongValue.class, pOrder.getType(1)); - assertEquals(StringValue.class, pOrder.getType(2)); - } - - @Test - public void testRangePartitioningErased() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"1;5"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - o.appendOrdering(2, StringValue.class, Order.ASCENDING); - GlobalProperties gprops = new GlobalProperties(); - gprops.setRangePartitioned(o); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.RANDOM_PARTITIONED, result.getPartitioning()); - assertNull(result.getPartitioningOrdering()); - assertNull(result.getPartitioningFields()); - } - - @Test - public void testRebalancingPreserved() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0->1; 1->2; 4->3"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setForcedRebalanced(); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - - assertEquals(PartitioningProperty.FORCED_REBALANCED, result.getPartitioning()); - assertNull(result.getPartitioningFields()); - } - - @Test - public void testUniqueFieldGroupsPreserved1() { - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1;2;3;4"}, null, null, tupleInfo, tupleInfo); - - FieldSet set1 = new FieldSet(0, 1, 2); - FieldSet set2 = new FieldSet(3, 4); - FieldSet set3 = new FieldSet(4, 5, 6, 7); - GlobalProperties gprops = new GlobalProperties(); - gprops.addUniqueFieldCombination(set1); - gprops.addUniqueFieldCombination(set2); - gprops.addUniqueFieldCombination(set3); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - Set
unique = result.getUniqueFieldCombination(); - FieldSet expected1 = new FieldSet(0, 1, 2); - FieldSet expected2 = new FieldSet(3, 4); - - Assert.assertTrue(unique.size() == 2); - Assert.assertTrue(unique.contains(expected1)); - Assert.assertTrue(unique.contains(expected2)); - } - - @Test - public void testUniqueFieldGroupsPreserved2() { - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0->5;1;2;3->6;4"}, null, null, tupleInfo, tupleInfo); - - FieldSet set1 = new FieldSet(0, 1, 2); - FieldSet set2 = new FieldSet(3, 4); - FieldSet set3 = new FieldSet(4, 5, 6, 7); - GlobalProperties gprops = new GlobalProperties(); - gprops.addUniqueFieldCombination(set1); - gprops.addUniqueFieldCombination(set2); - gprops.addUniqueFieldCombination(set3); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - Set
unique = result.getUniqueFieldCombination(); - FieldSet expected1 = new FieldSet(1, 2, 5); - FieldSet expected2 = new FieldSet(4, 6); - - Assert.assertTrue(unique.size() == 2); - Assert.assertTrue(unique.contains(expected1)); - Assert.assertTrue(unique.contains(expected2)); - } - - @Test - public void testUniqueFieldGroupsErased() { - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0; 3; 5; 6; 7"}, null, null, tupleInfo, tupleInfo); - - FieldSet set1 = new FieldSet(0, 1, 2); - FieldSet set2 = new FieldSet(3, 4); - FieldSet set3 = new FieldSet(4, 5, 6, 7); - GlobalProperties gprops = new GlobalProperties(); - gprops.addUniqueFieldCombination(set1); - gprops.addUniqueFieldCombination(set2); - gprops.addUniqueFieldCombination(set3); - - GlobalProperties result = gprops.filterBySemanticProperties(sprops, 0); - Assert.assertNull(result.getUniqueFieldCombination()); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testInvalidInputIndex() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1"}, null, null, tupleInfo, tupleInfo); - - GlobalProperties gprops = new GlobalProperties(); - gprops.setHashPartitioned(new FieldList(0, 1)); - - gprops.filterBySemanticProperties(sprops, 1); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java deleted file mode 100644 index 2de5ff0ec20c3..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java +++ /dev/null @@ -1,313 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.java.tuple.Tuple2; - -import org.junit.Test; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class GlobalPropertiesMatchingTest { - - @Test - public void testMatchingAnyPartitioning() { - try { - - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setAnyPartitioning(new FieldSet(6, 2)); - - // match any partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setAnyPartitioning(new FieldList(2, 6)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setAnyPartitioning(new FieldList(6, 2)); - assertTrue(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setAnyPartitioning(new FieldList(6, 2, 4)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setAnyPartitioning(new FieldList(6, 1)); - assertFalse(req.isMetBy(gp4)); - - GlobalProperties gp5 = new GlobalProperties(); - gp5.setAnyPartitioning(new FieldList(2)); - assertTrue(req.isMetBy(gp5)); - } - - // match hash partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setHashPartitioned(new FieldList(2, 6)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setHashPartitioned(new FieldList(6, 2)); - assertTrue(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setHashPartitioned(new FieldList(6, 1)); - assertFalse(req.isMetBy(gp3)); - } - - // match range partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setRangePartitioned( - new Ordering(2, null, Order.DESCENDING) - .appendOrdering(6, null, Order.ASCENDING)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setRangePartitioned( - new Ordering(6, null, Order.DESCENDING) - .appendOrdering(2, null, Order.ASCENDING)); - assertTrue(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setRangePartitioned( - new Ordering(6, null, Order.DESCENDING) - .appendOrdering(1, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setRangePartitioned(new Ordering(6, null, Order.DESCENDING)); - assertTrue(req.isMetBy(gp4)); - } - - // match custom partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setCustomPartitioned(new FieldList(2, 6), new MockPartitioner()); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setCustomPartitioned(new FieldList(6, 2), new MockPartitioner()); - assertTrue(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setCustomPartitioned(new FieldList(6, 1), new MockPartitioner()); - assertFalse(req.isMetBy(gp3)); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testMatchingCustomPartitioning() { - try { - final Partitioner> partitioner = new MockPartitioner(); - - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setCustomPartitioned(new FieldSet(6, 2), partitioner); - - // match custom partitionings - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setCustomPartitioned(new FieldList(2, 6), partitioner); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setCustomPartitioned(new FieldList(6, 2), partitioner); - assertTrue(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setCustomPartitioned(new FieldList(6, 2), new MockPartitioner()); - assertFalse(req.isMetBy(gp3)); - } - - // cannot match other types of partitionings - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setAnyPartitioning(new FieldList(6, 2)); - assertFalse(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setHashPartitioned(new FieldList(6, 2)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setRangePartitioned( - new Ordering(2, null, Order.DESCENDING) - .appendOrdering(6, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp3)); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testStrictlyMatchingAnyPartitioning() { - - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setAnyPartitioning(new FieldList(6, 2)); - - // match any partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setAnyPartitioning(new FieldList(6, 2)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setAnyPartitioning(new FieldList(2, 6)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setAnyPartitioning(new FieldList(6, 2, 3)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setAnyPartitioning(new FieldList(6, 1)); - assertFalse(req.isMetBy(gp4)); - - GlobalProperties gp5 = new GlobalProperties(); - gp5.setAnyPartitioning(new FieldList(2)); - assertFalse(req.isMetBy(gp5)); - } - - // match hash partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setHashPartitioned(new FieldList(6, 2)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setHashPartitioned(new FieldList(2, 6)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setHashPartitioned(new FieldList(6, 1)); - assertFalse(req.isMetBy(gp3)); - } - - // match range partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setRangePartitioned( - new Ordering(6, null, Order.DESCENDING) - .appendOrdering(2, null, Order.ASCENDING)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setRangePartitioned( - new Ordering(2, null, Order.DESCENDING) - .appendOrdering(6, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setRangePartitioned( - new Ordering(6, null, Order.DESCENDING) - .appendOrdering(1, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setRangePartitioned(new Ordering(6, null, Order.DESCENDING)); - assertFalse(req.isMetBy(gp4)); - } - } - - @Test - public void testStrictlyMatchingHashPartitioning() { - - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setHashPartitioned(new FieldList(6, 2)); - - // match any partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setAnyPartitioning(new FieldList(6, 2)); - assertFalse(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setAnyPartitioning(new FieldList(2, 6)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setAnyPartitioning(new FieldList(6, 1)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setAnyPartitioning(new FieldList(2)); - assertFalse(req.isMetBy(gp4)); - } - - // match hash partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setHashPartitioned(new FieldList(6, 2)); - assertTrue(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setHashPartitioned(new FieldList(2, 6)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setHashPartitioned(new FieldList(6, 1)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setHashPartitioned(new FieldList(6, 2, 0)); - assertFalse(req.isMetBy(gp4)); - } - - // match range partitioning - { - GlobalProperties gp1 = new GlobalProperties(); - gp1.setRangePartitioned( - new Ordering(6, null, Order.DESCENDING) - .appendOrdering(2, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp1)); - - GlobalProperties gp2 = new GlobalProperties(); - gp2.setRangePartitioned( - new Ordering(2, null, Order.DESCENDING) - .appendOrdering(6, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp2)); - - GlobalProperties gp3 = new GlobalProperties(); - gp3.setRangePartitioned( - new Ordering(6, null, Order.DESCENDING) - .appendOrdering(1, null, Order.ASCENDING)); - assertFalse(req.isMetBy(gp3)); - - GlobalProperties gp4 = new GlobalProperties(); - gp4.setRangePartitioned(new Ordering(6, null, Order.DESCENDING)); - assertFalse(req.isMetBy(gp4)); - } - } - - // -------------------------------------------------------------------------------------------- - -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesPushdownTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesPushdownTest.java deleted file mode 100644 index 12054a0afea25..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesPushdownTest.java +++ /dev/null @@ -1,110 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.operators.SemanticProperties; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; - -import org.junit.Test; - -import static org.junit.Assert.*; - -public class GlobalPropertiesPushdownTest { - - @Test - public void testAnyPartitioningPushedDown() { - try { - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setAnyPartitioning(new FieldSet(3, 1)); - - RequestedGlobalProperties preserved = - req.filterBySemanticProperties(getAllPreservingSemProps(), 0); - assertEquals(PartitioningProperty.ANY_PARTITIONING, preserved.getPartitioning()); - assertTrue(preserved.getPartitionedFields().isValidSubset(new FieldSet(1, 3))); - - RequestedGlobalProperties nonPreserved = - req.filterBySemanticProperties(getNonePreservingSemProps(), 0); - assertTrue(nonPreserved == null || nonPreserved.isTrivial()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testHashPartitioningPushedDown() { - try { - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setHashPartitioned(new FieldSet(3, 1)); - - RequestedGlobalProperties preserved = - req.filterBySemanticProperties(getAllPreservingSemProps(), 0); - assertEquals(PartitioningProperty.HASH_PARTITIONED, preserved.getPartitioning()); - assertTrue(preserved.getPartitionedFields().isValidSubset(new FieldSet(1, 3))); - - RequestedGlobalProperties nonPreserved = - req.filterBySemanticProperties(getNonePreservingSemProps(), 0); - assertTrue(nonPreserved == null || nonPreserved.isTrivial()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCustomPartitioningNotPushedDown() { - try { - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setCustomPartitioned(new FieldSet(3, 1), new MockPartitioner()); - - RequestedGlobalProperties pushedDown = - req.filterBySemanticProperties(getAllPreservingSemProps(), 0); - assertTrue(pushedDown == null || pushedDown.isTrivial()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testForcedReblancingNotPushedDown() { - try { - RequestedGlobalProperties req = new RequestedGlobalProperties(); - req.setForceRebalancing(); - - RequestedGlobalProperties pushedDown = - req.filterBySemanticProperties(getAllPreservingSemProps(), 0); - assertTrue(pushedDown == null || pushedDown.isTrivial()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static SemanticProperties getAllPreservingSemProps() { - return new SingleInputSemanticProperties.AllFieldsForwardedProperties(); - } - - private static SemanticProperties getNonePreservingSemProps() { - return new SingleInputSemanticProperties(); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/LocalPropertiesFilteringTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/LocalPropertiesFilteringTest.java deleted file mode 100644 index bf25ad973c9cd..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/LocalPropertiesFilteringTest.java +++ /dev/null @@ -1,404 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.junit.Test; - -import static org.junit.Assert.*; - -public class LocalPropertiesFilteringTest { - - private TupleTypeInfo< - Tuple8> - tupleInfo = - new TupleTypeInfo< - Tuple8< - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer>>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test - public void testAllErased1() { - - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, null, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = LocalProperties.forGrouping(new FieldList(0, 1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(3, 4)); - lProps = lProps.addUniqueFields(new FieldSet(5, 6)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - - assertNull(filtered.getGroupedFields()); - assertNull(filtered.getOrdering()); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testAllErased2() { - - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"5"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = LocalProperties.forGrouping(new FieldList(0, 1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(3, 4)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - - assertNull(filtered.getGroupedFields()); - assertNull(filtered.getOrdering()); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testGroupingPreserved1() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;2;3"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = LocalProperties.forGrouping(new FieldList(0, 2, 3)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - - assertNotNull(filtered.getGroupedFields()); - assertEquals(3, filtered.getGroupedFields().size()); - assertTrue(filtered.getGroupedFields().contains(0)); - assertTrue(filtered.getGroupedFields().contains(2)); - assertTrue(filtered.getGroupedFields().contains(3)); - assertNull(filtered.getOrdering()); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testGroupingPreserved2() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0->4;2->0;3->7"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = LocalProperties.forGrouping(new FieldList(0, 2, 3)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - - assertNotNull(filtered.getGroupedFields()); - assertEquals(3, filtered.getGroupedFields().size()); - assertTrue(filtered.getGroupedFields().contains(4)); - assertTrue(filtered.getGroupedFields().contains(0)); - assertTrue(filtered.getGroupedFields().contains(7)); - assertNull(filtered.getOrdering()); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testGroupingErased() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0->4;2->0"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = LocalProperties.forGrouping(new FieldList(0, 2, 3)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - - assertNull(filtered.getGroupedFields()); - assertNull(filtered.getOrdering()); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testSortingPreserved1() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;2;5"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(2, IntValue.class, Order.ASCENDING); - o.appendOrdering(0, StringValue.class, Order.DESCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - LocalProperties lProps = LocalProperties.forOrdering(o); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldList gFields = filtered.getGroupedFields(); - Ordering order = filtered.getOrdering(); - - assertNotNull(gFields); - assertEquals(3, gFields.size()); - assertTrue(gFields.contains(0)); - assertTrue(gFields.contains(2)); - assertTrue(gFields.contains(5)); - assertNotNull(order); - assertEquals(3, order.getNumberOfFields()); - assertEquals(2, order.getFieldNumber(0).intValue()); - assertEquals(0, order.getFieldNumber(1).intValue()); - assertEquals(5, order.getFieldNumber(2).intValue()); - assertEquals(Order.ASCENDING, order.getOrder(0)); - assertEquals(Order.DESCENDING, order.getOrder(1)); - assertEquals(Order.DESCENDING, order.getOrder(2)); - assertEquals(IntValue.class, order.getType(0)); - assertEquals(StringValue.class, order.getType(1)); - assertEquals(LongValue.class, order.getType(2)); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testSortingPreserved2() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0->3;2->7;5->1"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(2, IntValue.class, Order.ASCENDING); - o.appendOrdering(0, StringValue.class, Order.DESCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - LocalProperties lProps = LocalProperties.forOrdering(o); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldList gFields = filtered.getGroupedFields(); - Ordering order = filtered.getOrdering(); - - assertNotNull(gFields); - assertEquals(3, gFields.size()); - assertTrue(gFields.contains(3)); - assertTrue(gFields.contains(7)); - assertTrue(gFields.contains(1)); - assertNotNull(order); - assertEquals(3, order.getNumberOfFields()); - assertEquals(7, order.getFieldNumber(0).intValue()); - assertEquals(3, order.getFieldNumber(1).intValue()); - assertEquals(1, order.getFieldNumber(2).intValue()); - assertEquals(Order.ASCENDING, order.getOrder(0)); - assertEquals(Order.DESCENDING, order.getOrder(1)); - assertEquals(Order.DESCENDING, order.getOrder(2)); - assertEquals(IntValue.class, order.getType(0)); - assertEquals(StringValue.class, order.getType(1)); - assertEquals(LongValue.class, order.getType(2)); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testSortingPreserved3() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;2"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(2, IntValue.class, Order.ASCENDING); - o.appendOrdering(0, StringValue.class, Order.DESCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - LocalProperties lProps = LocalProperties.forOrdering(o); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldList gFields = filtered.getGroupedFields(); - Ordering order = filtered.getOrdering(); - - assertNotNull(gFields); - assertEquals(2, gFields.size()); - assertTrue(gFields.contains(0)); - assertTrue(gFields.contains(2)); - assertNotNull(order); - assertEquals(2, order.getNumberOfFields()); - assertEquals(2, order.getFieldNumber(0).intValue()); - assertEquals(0, order.getFieldNumber(1).intValue()); - assertEquals(Order.ASCENDING, order.getOrder(0)); - assertEquals(Order.DESCENDING, order.getOrder(1)); - assertEquals(IntValue.class, order.getType(0)); - assertEquals(StringValue.class, order.getType(1)); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testSortingPreserved4() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"2->7;5"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(2, IntValue.class, Order.ASCENDING); - o.appendOrdering(0, StringValue.class, Order.DESCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - LocalProperties lProps = LocalProperties.forOrdering(o); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldList gFields = filtered.getGroupedFields(); - Ordering order = filtered.getOrdering(); - - assertNotNull(gFields); - assertEquals(1, gFields.size()); - assertTrue(gFields.contains(7)); - assertNotNull(order); - assertEquals(1, order.getNumberOfFields()); - assertEquals(7, order.getFieldNumber(0).intValue()); - assertEquals(Order.ASCENDING, order.getOrder(0)); - assertEquals(IntValue.class, order.getType(0)); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testSortingErased() { - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;5"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(2, IntValue.class, Order.ASCENDING); - o.appendOrdering(0, StringValue.class, Order.DESCENDING); - o.appendOrdering(5, LongValue.class, Order.DESCENDING); - LocalProperties lProps = LocalProperties.forOrdering(o); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldList gFields = filtered.getGroupedFields(); - Ordering order = filtered.getOrdering(); - - assertNull(gFields); - assertNull(order); - assertNull(filtered.getUniqueFields()); - } - - @Test - public void testUniqueFieldsPreserved1() { - - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;1;2;3;4"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = new LocalProperties(); - lProps = lProps.addUniqueFields(new FieldSet(0, 1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(3, 4)); - lProps = lProps.addUniqueFields(new FieldSet(4, 5, 6)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldSet expected1 = new FieldSet(0, 1, 2); - FieldSet expected2 = new FieldSet(3, 4); - - assertNull(filtered.getGroupedFields()); - assertNull(filtered.getOrdering()); - assertNotNull(filtered.getUniqueFields()); - assertEquals(2, filtered.getUniqueFields().size()); - assertTrue(filtered.getUniqueFields().contains(expected1)); - assertTrue(filtered.getUniqueFields().contains(expected2)); - } - - @Test - public void testUniqueFieldsPreserved2() { - - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;1;2;3;4"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = LocalProperties.forGrouping(new FieldList(1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(0, 1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(3, 4)); - lProps = lProps.addUniqueFields(new FieldSet(4, 5, 6)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldSet expected1 = new FieldSet(0, 1, 2); - FieldSet expected2 = new FieldSet(3, 4); - - assertNull(filtered.getOrdering()); - assertNotNull(filtered.getGroupedFields()); - assertEquals(2, filtered.getGroupedFields().size()); - assertTrue(filtered.getGroupedFields().contains(1)); - assertTrue(filtered.getGroupedFields().contains(2)); - assertNotNull(filtered.getUniqueFields()); - assertEquals(2, filtered.getUniqueFields().size()); - assertTrue(filtered.getUniqueFields().contains(expected1)); - assertTrue(filtered.getUniqueFields().contains(expected2)); - } - - @Test - public void testUniqueFieldsPreserved3() { - - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0->7;1->6;2->5;3->4;4->3"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = new LocalProperties(); - lProps = lProps.addUniqueFields(new FieldSet(0, 1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(3, 4)); - lProps = lProps.addUniqueFields(new FieldSet(4, 5, 6)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - FieldSet expected1 = new FieldSet(5, 6, 7); - FieldSet expected2 = new FieldSet(3, 4); - - assertNull(filtered.getGroupedFields()); - assertNull(filtered.getOrdering()); - assertNotNull(filtered.getUniqueFields()); - assertEquals(2, filtered.getUniqueFields().size()); - assertTrue(filtered.getUniqueFields().contains(expected1)); - assertTrue(filtered.getUniqueFields().contains(expected2)); - } - - @Test - public void testUniqueFieldsErased() { - - SingleInputSemanticProperties sp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sp, new String[] {"0;1;4"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lProps = new LocalProperties(); - lProps = lProps.addUniqueFields(new FieldSet(0, 1, 2)); - lProps = lProps.addUniqueFields(new FieldSet(3, 4)); - lProps = lProps.addUniqueFields(new FieldSet(4, 5, 6)); - - LocalProperties filtered = lProps.filterBySemanticProperties(sp, 0); - - assertNull(filtered.getGroupedFields()); - assertNull(filtered.getOrdering()); - assertNull(filtered.getUniqueFields()); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testInvalidInputIndex() { - - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1"}, null, null, tupleInfo, tupleInfo); - - LocalProperties lprops = LocalProperties.forGrouping(new FieldList(0, 1)); - - lprops.filterBySemanticProperties(sprops, 1); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/MockDistribution.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/MockDistribution.java deleted file mode 100644 index 43f3700a4d659..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/MockDistribution.java +++ /dev/null @@ -1,52 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; - -@SuppressWarnings("serial") -public class MockDistribution implements DataDistribution { - - @Override - public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - return new Object[0]; - } - - @Override - public int getNumberOfFields() { - return 0; - } - - @Override - public TypeInformation[] getKeyTypes() { - return new TypeInformation[] {BasicTypeInfo.INT_TYPE_INFO}; - } - - @Override - public void write(DataOutputView out) throws IOException {} - - @Override - public void read(DataInputView in) throws IOException {} -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/MockPartitioner.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/MockPartitioner.java deleted file mode 100644 index 770772e6777e7..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/MockPartitioner.java +++ /dev/null @@ -1,32 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.java.tuple.Tuple2; - -class MockPartitioner implements Partitioner> { - - private static final long serialVersionUID = 1L; - - @Override - public int partition(Tuple2 key, int numPartitions) { - return 0; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalPropertiesFilteringTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalPropertiesFilteringTest.java deleted file mode 100644 index d848186463016..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/RequestedGlobalPropertiesFilteringTest.java +++ /dev/null @@ -1,470 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.types.ByteValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; - -import org.junit.Test; - -import static org.junit.Assert.*; - -public class RequestedGlobalPropertiesFilteringTest { - - private TupleTypeInfo< - Tuple8> - tupleInfo = - new TupleTypeInfo< - Tuple8< - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer>>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test(expected = NullPointerException.class) - public void testNullProps() { - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setAnyPartitioning(new FieldSet(0, 1, 2)); - - rgProps.filterBySemanticProperties(null, 0); - } - - @Test - public void testEraseAll1() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setAnyPartitioning(new FieldSet(0, 1, 2)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testEraseAll2() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"3;4"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setAnyPartitioning(new FieldSet(0, 1, 2)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testHashPartitioningPreserved1() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"0;3;4"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setHashPartitioned(new FieldSet(0, 3, 4)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.HASH_PARTITIONED, filtered.getPartitioning()); - assertNotNull(filtered.getPartitionedFields()); - assertEquals(3, filtered.getPartitionedFields().size()); - assertTrue(filtered.getPartitionedFields().contains(0)); - assertTrue(filtered.getPartitionedFields().contains(3)); - assertTrue(filtered.getPartitionedFields().contains(4)); - assertNull(filtered.getDataDistribution()); - assertNull(filtered.getCustomPartitioner()); - assertNull(filtered.getOrdering()); - } - - @Test - public void testHashPartitioningPreserved2() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"2->0;1->3;7->4"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setHashPartitioned(new FieldSet(0, 3, 4)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.HASH_PARTITIONED, filtered.getPartitioning()); - assertNotNull(filtered.getPartitionedFields()); - assertEquals(3, filtered.getPartitionedFields().size()); - assertTrue(filtered.getPartitionedFields().contains(1)); - assertTrue(filtered.getPartitionedFields().contains(2)); - assertTrue(filtered.getPartitionedFields().contains(7)); - assertNull(filtered.getDataDistribution()); - assertNull(filtered.getCustomPartitioner()); - assertNull(filtered.getOrdering()); - } - - @Test - public void testHashPartitioningErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"1;2"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setHashPartitioned(new FieldSet(0, 3, 4)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testAnyPartitioningPreserved1() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"0;3;4"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setAnyPartitioning(new FieldSet(0, 3, 4)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.ANY_PARTITIONING, filtered.getPartitioning()); - assertNotNull(filtered.getPartitionedFields()); - assertEquals(3, filtered.getPartitionedFields().size()); - assertTrue(filtered.getPartitionedFields().contains(0)); - assertTrue(filtered.getPartitionedFields().contains(3)); - assertTrue(filtered.getPartitionedFields().contains(4)); - assertNull(filtered.getDataDistribution()); - assertNull(filtered.getCustomPartitioner()); - assertNull(filtered.getOrdering()); - } - - @Test - public void testAnyPartitioningPreserved2() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"2->0;1->3;7->4"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setAnyPartitioning(new FieldSet(0, 3, 4)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.ANY_PARTITIONING, filtered.getPartitioning()); - assertNotNull(filtered.getPartitionedFields()); - assertEquals(3, filtered.getPartitionedFields().size()); - assertTrue(filtered.getPartitionedFields().contains(1)); - assertTrue(filtered.getPartitionedFields().contains(2)); - assertTrue(filtered.getPartitionedFields().contains(7)); - assertNull(filtered.getDataDistribution()); - assertNull(filtered.getCustomPartitioner()); - assertNull(filtered.getOrdering()); - } - - @Test - public void testAnyPartitioningErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"1;2"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setAnyPartitioning(new FieldSet(0, 3, 4)); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testRangePartitioningPreserved1() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"1;3;6"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(3, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setRangePartitioned(o); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.RANGE_PARTITIONED, filtered.getPartitioning()); - assertNotNull(filtered.getOrdering()); - assertEquals(3, filtered.getOrdering().getNumberOfFields()); - assertEquals(3, filtered.getOrdering().getFieldNumber(0).intValue()); - assertEquals(1, filtered.getOrdering().getFieldNumber(1).intValue()); - assertEquals(6, filtered.getOrdering().getFieldNumber(2).intValue()); - assertEquals(LongValue.class, filtered.getOrdering().getType(0)); - assertEquals(IntValue.class, filtered.getOrdering().getType(1)); - assertEquals(ByteValue.class, filtered.getOrdering().getType(2)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(0)); - assertEquals(Order.ASCENDING, filtered.getOrdering().getOrder(1)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(2)); - assertNull(filtered.getPartitionedFields()); - assertNull(filtered.getDataDistribution()); - assertNull(filtered.getCustomPartitioner()); - } - - @Test - public void testRangePartitioningPreserved2() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"7->3;1->1;2->6"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(3, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setRangePartitioned(o); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.RANGE_PARTITIONED, filtered.getPartitioning()); - assertNotNull(filtered.getOrdering()); - assertEquals(3, filtered.getOrdering().getNumberOfFields()); - assertEquals(7, filtered.getOrdering().getFieldNumber(0).intValue()); - assertEquals(1, filtered.getOrdering().getFieldNumber(1).intValue()); - assertEquals(2, filtered.getOrdering().getFieldNumber(2).intValue()); - assertEquals(LongValue.class, filtered.getOrdering().getType(0)); - assertEquals(IntValue.class, filtered.getOrdering().getType(1)); - assertEquals(ByteValue.class, filtered.getOrdering().getType(2)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(0)); - assertEquals(Order.ASCENDING, filtered.getOrdering().getOrder(1)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(2)); - assertNull(filtered.getPartitionedFields()); - assertNull(filtered.getDataDistribution()); - assertNull(filtered.getCustomPartitioner()); - } - - @Test - public void testRangePartitioningPreserved3() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"7->3;1->1;2->6"}, null, null, tupleInfo, tupleInfo); - - DataDistribution dd = new MockDistribution(); - Ordering o = new Ordering(); - o.appendOrdering(3, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setRangePartitioned(o, dd); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNotNull(filtered); - assertEquals(PartitioningProperty.RANGE_PARTITIONED, filtered.getPartitioning()); - assertNotNull(filtered.getOrdering()); - assertEquals(3, filtered.getOrdering().getNumberOfFields()); - assertEquals(7, filtered.getOrdering().getFieldNumber(0).intValue()); - assertEquals(1, filtered.getOrdering().getFieldNumber(1).intValue()); - assertEquals(2, filtered.getOrdering().getFieldNumber(2).intValue()); - assertEquals(LongValue.class, filtered.getOrdering().getType(0)); - assertEquals(IntValue.class, filtered.getOrdering().getType(1)); - assertEquals(ByteValue.class, filtered.getOrdering().getType(2)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(0)); - assertEquals(Order.ASCENDING, filtered.getOrdering().getOrder(1)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(2)); - assertNotNull(filtered.getDataDistribution()); - assertEquals(dd, filtered.getDataDistribution()); - assertNull(filtered.getPartitionedFields()); - assertNull(filtered.getCustomPartitioner()); - } - - @Test - public void testRangePartitioningErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"1;2"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(3, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setRangePartitioned(o); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testCustomPartitioningErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"0;1;2"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setCustomPartitioned(new FieldSet(0, 1, 2), new MockPartitioner()); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testRandomDistributionErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"0;1;2"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setRandomPartitioning(); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testReplicationErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"0;1;2"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setFullyReplicated(); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testRebalancingErased() { - - SingleInputSemanticProperties sProp = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProp, new String[] {"0;1;2"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties rgProps = new RequestedGlobalProperties(); - rgProps.setForceRebalancing(); - - RequestedGlobalProperties filtered = rgProps.filterBySemanticProperties(sProp, 0); - - assertNull(filtered); - } - - @Test - public void testDualHashPartitioningPreserved() { - - DualInputSemanticProperties dprops = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dprops, - new String[] {"0;2;4"}, - new String[] {"1->3;4->6;3->7"}, - null, - null, - null, - null, - tupleInfo, - tupleInfo, - tupleInfo); - - RequestedGlobalProperties gprops1 = new RequestedGlobalProperties(); - RequestedGlobalProperties gprops2 = new RequestedGlobalProperties(); - gprops1.setHashPartitioned(new FieldSet(2, 0, 4)); - gprops2.setHashPartitioned(new FieldSet(3, 6, 7)); - RequestedGlobalProperties filtered1 = gprops1.filterBySemanticProperties(dprops, 0); - RequestedGlobalProperties filtered2 = gprops2.filterBySemanticProperties(dprops, 1); - - assertNotNull(filtered1); - assertEquals(PartitioningProperty.HASH_PARTITIONED, filtered1.getPartitioning()); - assertNotNull(filtered1.getPartitionedFields()); - assertEquals(3, filtered1.getPartitionedFields().size()); - assertTrue(filtered1.getPartitionedFields().contains(0)); - assertTrue(filtered1.getPartitionedFields().contains(2)); - assertTrue(filtered1.getPartitionedFields().contains(4)); - assertNull(filtered1.getOrdering()); - assertNull(filtered1.getCustomPartitioner()); - assertNull(filtered1.getDataDistribution()); - - assertNotNull(filtered2); - assertEquals(PartitioningProperty.HASH_PARTITIONED, filtered2.getPartitioning()); - assertNotNull(filtered2.getPartitionedFields()); - assertEquals(3, filtered2.getPartitionedFields().size()); - assertTrue(filtered2.getPartitionedFields().contains(1)); - assertTrue(filtered2.getPartitionedFields().contains(3)); - assertTrue(filtered2.getPartitionedFields().contains(4)); - assertNull(filtered2.getOrdering()); - assertNull(filtered2.getCustomPartitioner()); - assertNull(filtered2.getDataDistribution()); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testInvalidInputIndex() { - SingleInputSemanticProperties sprops = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sprops, new String[] {"0;1"}, null, null, tupleInfo, tupleInfo); - - RequestedGlobalProperties gprops = new RequestedGlobalProperties(); - gprops.setHashPartitioned(new FieldSet(0, 1)); - - gprops.filterBySemanticProperties(sprops, 1); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/RequestedLocalPropertiesFilteringTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/RequestedLocalPropertiesFilteringTest.java deleted file mode 100644 index 47a8c574aba4e..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/RequestedLocalPropertiesFilteringTest.java +++ /dev/null @@ -1,278 +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.optimizer.dataproperties; - -import org.apache.flink.api.common.operators.DualInputSemanticProperties; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.SingleInputSemanticProperties; -import org.apache.flink.api.common.operators.util.FieldSet; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.types.ByteValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; - -import org.junit.Test; - -import static org.junit.Assert.*; - -public class RequestedLocalPropertiesFilteringTest { - - private TupleTypeInfo< - Tuple8> - tupleInfo = - new TupleTypeInfo< - Tuple8< - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer, - Integer>>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - @Test(expected = NullPointerException.class) - public void testNullProps() { - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setGroupedFields(new FieldSet(0, 2, 3)); - - rlProp.filterBySemanticProperties(null, 0); - } - - @Test - public void testAllErased() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setGroupedFields(new FieldSet(0, 2, 3)); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNull(filtered); - } - - @Test - public void testGroupingPreserved1() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"0;2;3"}, null, null, tupleInfo, tupleInfo); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setGroupedFields(new FieldSet(0, 2, 3)); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNotNull(filtered); - assertNotNull(filtered.getGroupedFields()); - assertEquals(3, filtered.getGroupedFields().size()); - assertTrue(filtered.getGroupedFields().contains(0)); - assertTrue(filtered.getGroupedFields().contains(2)); - assertTrue(filtered.getGroupedFields().contains(3)); - assertNull(filtered.getOrdering()); - } - - @Test - public void testGroupingPreserved2() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"3->0;5->2;1->3"}, null, null, tupleInfo, tupleInfo); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setGroupedFields(new FieldSet(0, 2, 3)); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNotNull(filtered); - assertNotNull(filtered.getGroupedFields()); - assertEquals(3, filtered.getGroupedFields().size()); - assertTrue(filtered.getGroupedFields().contains(3)); - assertTrue(filtered.getGroupedFields().contains(5)); - assertTrue(filtered.getGroupedFields().contains(1)); - assertNull(filtered.getOrdering()); - } - - @Test - public void testGroupingErased() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"0;2"}, null, null, tupleInfo, tupleInfo); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setGroupedFields(new FieldSet(0, 2, 3)); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNull(filtered); - } - - @Test - public void testOrderPreserved1() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"1;4;6"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(4, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setOrdering(o); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNotNull(filtered); - assertNotNull(filtered.getOrdering()); - assertEquals(3, filtered.getOrdering().getNumberOfFields()); - assertEquals(4, filtered.getOrdering().getFieldNumber(0).intValue()); - assertEquals(1, filtered.getOrdering().getFieldNumber(1).intValue()); - assertEquals(6, filtered.getOrdering().getFieldNumber(2).intValue()); - assertEquals(LongValue.class, filtered.getOrdering().getType(0)); - assertEquals(IntValue.class, filtered.getOrdering().getType(1)); - assertEquals(ByteValue.class, filtered.getOrdering().getType(2)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(0)); - assertEquals(Order.ASCENDING, filtered.getOrdering().getOrder(1)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(2)); - assertNull(filtered.getGroupedFields()); - } - - @Test - public void testOrderPreserved2() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"5->1;0->4;2->6"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(4, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setOrdering(o); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNotNull(filtered); - assertNotNull(filtered.getOrdering()); - assertEquals(3, filtered.getOrdering().getNumberOfFields()); - assertEquals(0, filtered.getOrdering().getFieldNumber(0).intValue()); - assertEquals(5, filtered.getOrdering().getFieldNumber(1).intValue()); - assertEquals(2, filtered.getOrdering().getFieldNumber(2).intValue()); - assertEquals(LongValue.class, filtered.getOrdering().getType(0)); - assertEquals(IntValue.class, filtered.getOrdering().getType(1)); - assertEquals(ByteValue.class, filtered.getOrdering().getType(2)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(0)); - assertEquals(Order.ASCENDING, filtered.getOrdering().getOrder(1)); - assertEquals(Order.DESCENDING, filtered.getOrdering().getOrder(2)); - assertNull(filtered.getGroupedFields()); - } - - @Test - public void testOrderErased() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"1; 4"}, null, null, tupleInfo, tupleInfo); - - Ordering o = new Ordering(); - o.appendOrdering(4, LongValue.class, Order.DESCENDING); - o.appendOrdering(1, IntValue.class, Order.ASCENDING); - o.appendOrdering(6, ByteValue.class, Order.DESCENDING); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setOrdering(o); - - RequestedLocalProperties filtered = rlProp.filterBySemanticProperties(sProps, 0); - - assertNull(filtered); - } - - @Test - public void testDualGroupingPreserved() { - - DualInputSemanticProperties dprops = new DualInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsDualFromString( - dprops, - new String[] {"1->0;3;2->4"}, - new String[] {"0->7;1"}, - null, - null, - null, - null, - tupleInfo, - tupleInfo, - tupleInfo); - - RequestedLocalProperties lprops1 = new RequestedLocalProperties(); - lprops1.setGroupedFields(new FieldSet(0, 3, 4)); - - RequestedLocalProperties lprops2 = new RequestedLocalProperties(); - lprops2.setGroupedFields(new FieldSet(7, 1)); - - RequestedLocalProperties filtered1 = lprops1.filterBySemanticProperties(dprops, 0); - RequestedLocalProperties filtered2 = lprops2.filterBySemanticProperties(dprops, 1); - - assertNotNull(filtered1); - assertNotNull(filtered1.getGroupedFields()); - assertEquals(3, filtered1.getGroupedFields().size()); - assertTrue(filtered1.getGroupedFields().contains(1)); - assertTrue(filtered1.getGroupedFields().contains(2)); - assertTrue(filtered1.getGroupedFields().contains(3)); - assertNull(filtered1.getOrdering()); - - assertNotNull(filtered2); - assertNotNull(filtered2.getGroupedFields()); - assertEquals(2, filtered2.getGroupedFields().size()); - assertTrue(filtered2.getGroupedFields().contains(0)); - assertTrue(filtered2.getGroupedFields().contains(1)); - assertNull(filtered2.getOrdering()); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testInvalidInputIndex() { - - SingleInputSemanticProperties sProps = new SingleInputSemanticProperties(); - SemanticPropUtil.getSemanticPropsSingleFromString( - sProps, new String[] {"1; 4"}, null, null, tupleInfo, tupleInfo); - - RequestedLocalProperties rlProp = new RequestedLocalProperties(); - rlProp.setGroupedFields(new FieldSet(1, 4)); - - rlProp.filterBySemanticProperties(sProps, 1); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java deleted file mode 100644 index 4ad9f02fd359c..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DeltaIterationDependenciesTest.java +++ /dev/null @@ -1,84 +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.optimizer.java; - -import org.apache.flink.api.common.Plan; -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.fail; - -@SuppressWarnings({"serial", "unchecked"}) -public class DeltaIterationDependenciesTest extends CompilerTestBase { - - @Test - public void testExceptionWhenNewWorksetNotDependentOnWorkset() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = env.fromElements(new Tuple2(0L, 0L)); - - DeltaIteration, Tuple2> deltaIteration = - input.iterateDelta(input, 10, 0); - - DataSet> delta = - deltaIteration - .getSolutionSet() - .join(deltaIteration.getWorkset()) - .where(0) - .equalTo(0) - .projectFirst(1) - .projectSecond(1); - - DataSet> nextWorkset = - deltaIteration - .getSolutionSet() - .join(input) - .where(0) - .equalTo(0) - .projectFirst(1) - .projectSecond(1); - - DataSet> result = deltaIteration.closeWith(delta, nextWorkset); - - result.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - try { - compileNoStats(p); - fail( - "Should not be able to compile, since the next workset does not depend on the workset"); - } catch (CompilerException e) { - // good - } catch (Exception e) { - fail("wrong exception type"); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java deleted file mode 100644 index 4bd57610d723a..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/DistinctAndGroupingOptimizerTest.java +++ /dev/null @@ -1,120 +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.optimizer.java; - -import org.apache.flink.api.common.Plan; -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.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class DistinctAndGroupingOptimizerTest extends CompilerTestBase { - - @Test - public void testDistinctPreservesPartitioningOfDistinctFields() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - @SuppressWarnings("unchecked") - DataSet> data = - env.fromElements(new Tuple2(0L, 0L), new Tuple2(1L, 1L)) - .map(new IdentityMapper>()) - .setParallelism(4); - - data.distinct(0) - .groupBy(0) - .sum(1) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode distinctReducer = - (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - // reducer can be forward, reuses partitioning from distinct - assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy()); - - // distinct reducer is partitioned - assertEquals( - ShipStrategyType.PARTITION_HASH, distinctReducer.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testDistinctDestroysPartitioningOfNonDistinctFields() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - @SuppressWarnings("unchecked") - DataSet> data = - env.fromElements(new Tuple2(0L, 0L), new Tuple2(1L, 1L)) - .map(new IdentityMapper>()) - .setParallelism(4); - - data.distinct(1) - .groupBy(0) - .sum(1) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getInput().getSource(); - SingleInputPlanNode distinctReducer = - (SingleInputPlanNode) combiner.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - // reducer must repartition, because it works on a different field - assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); - - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - - // distinct reducer is partitioned - assertEquals( - ShipStrategyType.PARTITION_HASH, distinctReducer.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java deleted file mode 100644 index 50dd98ee73759..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/GroupReduceCompilationTest.java +++ /dev/null @@ -1,421 +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.optimizer.java; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.common.operators.util.FieldList; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.GroupReduceOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.util.Collector; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class GroupReduceCompilationTest extends CompilerTestBase implements java.io.Serializable { - - @Test - public void testAllGroupReduceNoCombiner() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source"); - - data.reduceGroup( - new RichGroupReduceFunction() { - public void reduce( - Iterable values, Collector out) {} - }) - .name("reducer") - .output(new DiscardingOutputFormat()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // the all-reduce has no combiner, when the parallelism of the input is one - - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // check wiring - assertEquals(sourceNode, reduceNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that reduce has the right strategy - assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy()); - - // check parallelism - assertEquals(1, sourceNode.getParallelism()); - assertEquals(1, reduceNode.getParallelism()); - assertEquals(1, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testAllReduceWithCombiner() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet data = env.generateSequence(1, 8000000).name("source"); - - GroupReduceOperator reduced = - data.reduceGroup(new CombineReducer2()).name("reducer"); - - reduced.setCombinable(true); - reduced.output(new DiscardingOutputFormat()).name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.ALL_GROUP_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.ALL_GROUP_REDUCE_COMBINE, combineNode.getDriverStrategy()); - - // check parallelism - assertEquals(8, sourceNode.getParallelism()); - assertEquals(8, combineNode.getParallelism()); - assertEquals(1, reduceNode.getParallelism()); - assertEquals(1, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithFieldPositionKeyNonCombinable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.groupBy(1) - .reduceGroup( - new RichGroupReduceFunction< - Tuple2, Tuple2>() { - public void reduce( - Iterable> values, - Collector> out) {} - }) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // check wiring - assertEquals(sourceNode, reduceNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reduceNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(1), reduceNode.getKeys(0)); - assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithFieldPositionKeyCombinable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - GroupReduceOperator, Tuple2> reduced = - data.groupBy(1).reduceGroup(new CombineReducer()).name("reducer"); - - reduced.setCombinable(true); - reduced.output(new DiscardingOutputFormat>()).name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(1), reduceNode.getKeys(0)); - assertEquals(new FieldList(1), combineNode.getKeys(0)); - assertEquals(new FieldList(1), combineNode.getKeys(1)); - assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithSelectorFunctionKeyNoncombinable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.groupBy( - new KeySelector, String>() { - public String getKey(Tuple2 value) { - return value.f0; - } - }) - .reduceGroup( - new RichGroupReduceFunction< - Tuple2, Tuple2>() { - public void reduce( - Iterable> values, - Collector> out) {} - }) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the key extractors and projectors - SingleInputPlanNode keyExtractor = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - SingleInputPlanNode keyProjector = - (SingleInputPlanNode) sinkNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, keyExtractor.getInput().getSource()); - assertEquals(keyProjector, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reduceNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0), reduceNode.getKeys(0)); - assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, keyExtractor.getParallelism()); - - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, keyProjector.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithSelectorFunctionKeyCombinable() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - GroupReduceOperator, Tuple2> reduced = - data.groupBy( - new KeySelector, String>() { - public String getKey(Tuple2 value) { - return value.f0; - } - }) - .reduceGroup(new CombineReducer()) - .name("reducer"); - - reduced.setCombinable(true); - reduced.output(new DiscardingOutputFormat>()).name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // get the key extractors and projectors - SingleInputPlanNode keyExtractor = - (SingleInputPlanNode) combineNode.getInput().getSource(); - SingleInputPlanNode keyProjector = - (SingleInputPlanNode) sinkNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, keyExtractor.getInput().getSource()); - assertEquals(keyProjector, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0), reduceNode.getKeys(0)); - assertEquals(new FieldList(0), combineNode.getKeys(0)); - assertEquals(new FieldList(0), combineNode.getKeys(1)); - assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, keyExtractor.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, keyProjector.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - public static class CombineReducer - implements GroupReduceFunction, Tuple2>, - GroupCombineFunction, Tuple2> { - - @Override - public void reduce( - Iterable> values, Collector> out) {} - - @Override - public void combine( - Iterable> values, Collector> out) {} - } - - public static class CombineReducer2 - implements GroupReduceFunction, GroupCombineFunction { - - @Override - public void reduce(Iterable values, Collector out) {} - - @Override - public void combine(Iterable values, Collector out) {} - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java deleted file mode 100644 index b925488412806..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/IterationCompilerTest.java +++ /dev/null @@ -1,206 +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.optimizer.java; - -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.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.NAryUnionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class IterationCompilerTest extends CompilerTestBase { - - @Test - public void testIdentityIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(43); - - IterativeDataSet iteration = env.generateSequence(-4, 1000).iterate(100); - iteration.closeWith(iteration).output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testEmptyWorksetIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(43); - - DataSet> input = - env.generateSequence(1, 20) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) { - return null; - } - }); - - DeltaIteration, Tuple2> iter = - input.iterateDelta(input, 100, 0); - iter.closeWith(iter.getWorkset(), iter.getWorkset()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testIterationWithUnionRoot() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(43); - - IterativeDataSet iteration = env.generateSequence(-4, 1000).iterate(100); - - iteration - .closeWith( - iteration - .map(new IdentityMapper()) - .union(iteration.map(new IdentityMapper()))) - .output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - BulkIterationPlanNode iterNode = (BulkIterationPlanNode) sink.getInput().getSource(); - - // make sure that the root is part of the dynamic path - - // the "NoOp" that comes after the union. - SingleInputPlanNode noop = (SingleInputPlanNode) iterNode.getRootOfStepFunction(); - NAryUnionPlanNode union = (NAryUnionPlanNode) noop.getInput().getSource(); - - assertTrue(noop.isOnDynamicPath()); - assertTrue(noop.getCostWeight() >= 1); - - assertTrue(union.isOnDynamicPath()); - assertTrue(union.getCostWeight() >= 1); - - // see that the jobgraph generator can translate this - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testWorksetIterationWithUnionRoot() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(43); - - DataSet> input = - env.generateSequence(1, 20) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) { - return null; - } - }); - - DeltaIteration, Tuple2> iter = - input.iterateDelta(input, 100, 0); - iter.closeWith( - iter.getWorkset() - .map(new IdentityMapper>()) - .union( - iter.getWorkset() - .map(new IdentityMapper>())), - iter.getWorkset() - .map(new IdentityMapper>()) - .union( - iter.getWorkset() - .map(new IdentityMapper>()))) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - WorksetIterationPlanNode iterNode = - (WorksetIterationPlanNode) sink.getInput().getSource(); - - // make sure that the root is part of the dynamic path - - // the "NoOp"a that come after the union. - SingleInputPlanNode nextWorksetNoop = - (SingleInputPlanNode) iterNode.getNextWorkSetPlanNode(); - SingleInputPlanNode solutionDeltaNoop = - (SingleInputPlanNode) iterNode.getSolutionSetDeltaPlanNode(); - - NAryUnionPlanNode nextWorksetUnion = - (NAryUnionPlanNode) nextWorksetNoop.getInput().getSource(); - NAryUnionPlanNode solutionDeltaUnion = - (NAryUnionPlanNode) solutionDeltaNoop.getInput().getSource(); - - assertTrue(nextWorksetNoop.isOnDynamicPath()); - assertTrue(nextWorksetNoop.getCostWeight() >= 1); - - assertTrue(solutionDeltaNoop.isOnDynamicPath()); - assertTrue(solutionDeltaNoop.getCostWeight() >= 1); - - assertTrue(nextWorksetUnion.isOnDynamicPath()); - assertTrue(nextWorksetUnion.getCostWeight() >= 1); - - assertTrue(solutionDeltaUnion.isOnDynamicPath()); - assertTrue(solutionDeltaUnion.getCostWeight() >= 1); - - new JobGraphGenerator().compileJobGraph(op); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java deleted file mode 100644 index 55de84387c2a5..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/JoinTranslationTest.java +++ /dev/null @@ -1,169 +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.optimizer.java; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Visitor; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class JoinTranslationTest extends CompilerTestBase { - - @Test - public void testBroadcastHashFirstTest() { - try { - DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.BROADCAST_HASH_FIRST); - assertEquals(ShipStrategyType.BROADCAST, node.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, node.getInput2().getShipStrategy()); - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, node.getDriverStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getClass().getSimpleName() + ": " + e.getMessage()); - } - } - - @Test - public void testBroadcastHashSecondTest() { - try { - DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.BROADCAST_HASH_SECOND); - assertEquals(ShipStrategyType.FORWARD, node.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.BROADCAST, node.getInput2().getShipStrategy()); - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, node.getDriverStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getClass().getSimpleName() + ": " + e.getMessage()); - } - } - - @Test - public void testPartitionHashFirstTest() { - try { - DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.REPARTITION_HASH_FIRST); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, node.getDriverStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getClass().getSimpleName() + ": " + e.getMessage()); - } - } - - @Test - public void testPartitionHashSecondTest() { - try { - DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.REPARTITION_HASH_SECOND); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, node.getDriverStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getClass().getSimpleName() + ": " + e.getMessage()); - } - } - - @Test - public void testPartitionSortMergeTest() { - try { - DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.REPARTITION_SORT_MERGE); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); - assertEquals(DriverStrategy.INNER_MERGE, node.getDriverStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getClass().getSimpleName() + ": " + e.getMessage()); - } - } - - @Test - public void testOptimizerChoosesTest() { - try { - DualInputPlanNode node = createPlanAndGetJoinNode(JoinHint.OPTIMIZER_CHOOSES); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput1().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, node.getInput2().getShipStrategy()); - assertTrue( - DriverStrategy.HYBRIDHASH_BUILD_FIRST == node.getDriverStrategy() - || DriverStrategy.HYBRIDHASH_BUILD_SECOND == node.getDriverStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getClass().getSimpleName() + ": " + e.getMessage()); - } - } - - private DualInputPlanNode createPlanAndGetJoinNode(JoinHint hint) { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet i1 = env.generateSequence(1, 1000); - DataSet i2 = env.generateSequence(1, 1000); - - i1.join(i2, hint) - .where(new IdentityKeySelector()) - .equalTo(new IdentityKeySelector()) - .output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - - // set statistics to the sources - plan.accept( - new Visitor>() { - @Override - public boolean preVisit(Operator visitable) { - if (visitable instanceof GenericDataSourceBase) { - GenericDataSourceBase source = - (GenericDataSourceBase) visitable; - setSourceStatistics(source, 10000000, 1000); - } - - return true; - } - - @Override - public void postVisit(Operator visitable) {} - }); - - OptimizedPlan op = compileWithStats(plan); - - return (DualInputPlanNode) - ((SinkPlanNode) op.getDataSinks().iterator().next()).getInput().getSource(); - } - - private static final class IdentityKeySelector implements KeySelector { - - @Override - public T getKey(T value) { - return value; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.java deleted file mode 100644 index 3514f06ae2373..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/OpenIterationTest.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.optimizer.java; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -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.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class OpenIterationTest extends CompilerTestBase { - - @Test - public void testSinkInOpenBulkIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input = env.generateSequence(1, 10); - - IterativeDataSet iteration = input.iterate(10); - - DataSet mapped = iteration.map(new IdentityMapper()); - - mapped.output(new DiscardingOutputFormat()); - - try { - env.createProgramPlan(); - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testSinkInClosedBulkIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input = env.generateSequence(1, 10); - - IterativeDataSet iteration = input.iterate(10); - - DataSet mapped = iteration.map(new IdentityMapper()); - - iteration.closeWith(mapped).output(new DiscardingOutputFormat()); - - mapped.output(new DiscardingOutputFormat()); - - Plan p = env.createProgramPlan(); - - try { - compileNoStats(p); - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testSinkOnSolutionSetDeltaIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple2(0L, 0L)); - - DeltaIteration, Tuple2> iteration = - input.iterateDelta(input, 10, 0); - - DataSet> mapped = - iteration.getSolutionSet().map(new IdentityMapper>()); - - mapped.output(new DiscardingOutputFormat>()); - - try { - env.createProgramPlan(); - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testSinkOnWorksetDeltaIteration() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple2(0L, 0L)); - - DeltaIteration, Tuple2> iteration = - input.iterateDelta(input, 10, 0); - - DataSet> mapped = - iteration.getWorkset().map(new IdentityMapper>()); - - mapped.output(new DiscardingOutputFormat>()); - - try { - env.createProgramPlan(); - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testOperationOnSolutionSet() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> input = env.fromElements(new Tuple2(0L, 0L)); - - DeltaIteration, Tuple2> iteration = - input.iterateDelta(input, 10, 0); - - DataSet> mapped = - iteration.getSolutionSet().map(new IdentityMapper>()); - - DataSet> joined = - iteration - .getWorkset() - .join(mapped) - .where(0) - .equalTo(0) - .projectFirst(1) - .projectSecond(0); - - iteration - .closeWith(joined, joined) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - try { - compileNoStats(p); - fail("should throw an exception"); - } catch (InvalidProgramException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java deleted file mode 100644 index f0db23f0cf660..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/PartitionOperatorTest.java +++ /dev/null @@ -1,215 +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.optimizer.java; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.Partitioner; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.PartitionOperator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.testfunctions.IdentityGroupReducerCombinable; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import java.util.Collections; -import java.util.List; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class PartitionOperatorTest extends CompilerTestBase { - - @Test - public void testPartitionCustomOperatorPreservesFields() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromCollection(Collections.singleton(new Tuple2<>(0L, 0L))); - - data.partitionCustom( - new Partitioner() { - public int partition(Long key, int numPartitions) { - return key.intValue(); - } - }, - 1) - .groupBy(1) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitioner = (SingleInputPlanNode) reducer.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, partitioner.getInput().getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testRangePartitionOperatorPreservesFields() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromCollection(Collections.singleton(new Tuple2<>(0L, 0L))); - - data.partitionByRange(1) - .groupBy(1) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitionNode = - (SingleInputPlanNode) reducer.getInput().getSource(); - SingleInputPlanNode partitionIDRemover = - (SingleInputPlanNode) partitionNode.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, partitionNode.getInput().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, - partitionIDRemover.getInput().getShipStrategy()); - - SourcePlanNode sourcePlanNode = op.getDataSources().iterator().next(); - List sourceOutgoingChannels = sourcePlanNode.getOutgoingChannels(); - assertEquals(2, sourceOutgoingChannels.size()); - assertEquals(ShipStrategyType.FORWARD, sourceOutgoingChannels.get(0).getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, sourceOutgoingChannels.get(1).getShipStrategy()); - assertEquals( - DataExchangeMode.PIPELINED, - sourceOutgoingChannels.get(0).getDataExchangeMode()); - assertEquals( - DataExchangeMode.BATCH, sourceOutgoingChannels.get(1).getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testRangePartitionOperatorPreservesFields2() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.fromCollection(Collections.singleton(new Tuple2<>(0L, 0L))); - - PartitionOperator> rangePartitioned = data.partitionByRange(1); - - rangePartitioned - .groupBy(1) - .reduceGroup(new IdentityGroupReducerCombinable>()) - .output(new DiscardingOutputFormat>()); - - data.groupBy(0) - .aggregate(Aggregations.SUM, 1) - .map( - new MapFunction, Long>() { - @Override - public Long map(Tuple2 value) throws Exception { - return value.f1; - } - }) - .output(new DiscardingOutputFormat()); - - rangePartitioned - .filter( - new FilterFunction>() { - @Override - public boolean filter(Tuple2 value) throws Exception { - return value.f0 % 2 == 0; - } - }) - .output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sink = op.getDataSinks().iterator().next(); - SingleInputPlanNode reducer = (SingleInputPlanNode) sink.getInput().getSource(); - SingleInputPlanNode partitionNode = - (SingleInputPlanNode) reducer.getInput().getSource(); - SingleInputPlanNode partitionIDRemover = - (SingleInputPlanNode) partitionNode.getInput().getSource(); - - assertEquals(ShipStrategyType.FORWARD, reducer.getInput().getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, partitionNode.getInput().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_CUSTOM, - partitionIDRemover.getInput().getShipStrategy()); - - SourcePlanNode sourcePlanNode = op.getDataSources().iterator().next(); - List sourceOutgoingChannels = sourcePlanNode.getOutgoingChannels(); - assertEquals(3, sourceOutgoingChannels.size()); - assertEquals(ShipStrategyType.FORWARD, sourceOutgoingChannels.get(0).getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, sourceOutgoingChannels.get(1).getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, sourceOutgoingChannels.get(2).getShipStrategy()); - assertEquals( - DataExchangeMode.PIPELINED, - sourceOutgoingChannels.get(0).getDataExchangeMode()); - assertEquals( - DataExchangeMode.PIPELINED, - sourceOutgoingChannels.get(1).getDataExchangeMode()); - assertEquals( - DataExchangeMode.BATCH, sourceOutgoingChannels.get(2).getDataExchangeMode()); - - List partitionOutputChannels = partitionNode.getOutgoingChannels(); - assertEquals(2, partitionOutputChannels.size()); - assertEquals( - ShipStrategyType.FORWARD, partitionOutputChannels.get(0).getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, partitionOutputChannels.get(1).getShipStrategy()); - assertEquals( - DataExchangeMode.PIPELINED, - partitionOutputChannels.get(0).getDataExchangeMode()); - assertEquals( - DataExchangeMode.PIPELINED, - partitionOutputChannels.get(1).getDataExchangeMode()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java deleted file mode 100644 index 61080157f8def..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/ReduceCompilationTest.java +++ /dev/null @@ -1,423 +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.optimizer.java; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichReduceFunction; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint; -import org.apache.flink.api.common.operators.util.FieldList; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -@SuppressWarnings("serial") -public class ReduceCompilationTest extends CompilerTestBase implements java.io.Serializable { - - @Test - public void testAllReduceNoCombiner() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet data = env.fromElements(0.2, 0.3, 0.4, 0.5).name("source"); - - data.reduce( - new RichReduceFunction() { - - @Override - public Double reduce(Double value1, Double value2) { - return value1 + value2; - } - }) - .name("reducer") - .output(new DiscardingOutputFormat()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // the all-reduce has no combiner, when the parallelism of the input is one - - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // check wiring - assertEquals(sourceNode, reduceNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check parallelism - assertEquals(1, sourceNode.getParallelism()); - assertEquals(1, reduceNode.getParallelism()); - assertEquals(1, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testAllReduceWithCombiner() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet data = env.generateSequence(1, 8000000).name("source"); - - data.reduce( - new RichReduceFunction() { - - @Override - public Long reduce(Long value1, Long value2) { - return value1 + value2; - } - }) - .name("reducer") - .output(new DiscardingOutputFormat()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check that both reduce and combiner have the same strategy - assertEquals(DriverStrategy.ALL_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.ALL_REDUCE, combineNode.getDriverStrategy()); - - // check parallelism - assertEquals(8, sourceNode.getParallelism()); - assertEquals(8, combineNode.getParallelism()); - assertEquals(1, reduceNode.getParallelism()); - assertEquals(1, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithFieldPositionKey() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.groupBy(1) - .reduce( - new RichReduceFunction>() { - @Override - public Tuple2 reduce( - Tuple2 value1, - Tuple2 value2) { - return null; - } - }) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, combineNode.getInput().getSource()); - assertEquals(reduceNode, sinkNode.getInput().getSource()); - - // check the strategies - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(1), reduceNode.getKeys(0)); - assertEquals(new FieldList(1), combineNode.getKeys(0)); - assertEquals(new FieldList(1), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithSelectorFunctionKey() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.groupBy( - new KeySelector, String>() { - public String getKey(Tuple2 value) { - return value.f0; - } - }) - .reduce( - new RichReduceFunction>() { - @Override - public Tuple2 reduce( - Tuple2 value1, - Tuple2 value2) { - return null; - } - }) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // get the key extractors and projectors - SingleInputPlanNode keyExtractor = - (SingleInputPlanNode) combineNode.getInput().getSource(); - SingleInputPlanNode keyProjector = - (SingleInputPlanNode) sinkNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, keyExtractor.getInput().getSource()); - assertEquals(keyProjector, sinkNode.getInput().getSource()); - - // check the strategies - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.SORTED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0), reduceNode.getKeys(0)); - assertEquals(new FieldList(0), combineNode.getKeys(0)); - assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, keyExtractor.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, keyProjector.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - @Test - public void testGroupedReduceWithHint() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.groupBy( - new KeySelector, String>() { - public String getKey(Tuple2 value) { - return value.f0; - } - }) - .reduce( - new RichReduceFunction>() { - @Override - public Tuple2 reduce( - Tuple2 value1, - Tuple2 value2) { - return null; - } - }) - .setCombineHint(CombineHint.HASH) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // get the combiner - SingleInputPlanNode combineNode = - (SingleInputPlanNode) reduceNode.getInput().getSource(); - - // get the key extractors and projectors - SingleInputPlanNode keyExtractor = - (SingleInputPlanNode) combineNode.getInput().getSource(); - SingleInputPlanNode keyProjector = - (SingleInputPlanNode) sinkNode.getInput().getSource(); - - // check wiring - assertEquals(sourceNode, keyExtractor.getInput().getSource()); - assertEquals(keyProjector, sinkNode.getInput().getSource()); - - // check the strategies - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - assertEquals(DriverStrategy.HASHED_PARTIAL_REDUCE, combineNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0), reduceNode.getKeys(0)); - assertEquals(new FieldList(0), combineNode.getKeys(0)); - assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(6, keyExtractor.getParallelism()); - assertEquals(6, combineNode.getParallelism()); - - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, keyProjector.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail(e.getClass().getSimpleName() + " in test: " + e.getMessage()); - } - } - - /** - * Test program compilation when the Reduce's combiner has been excluded by setting {@code - * CombineHint.NONE}. - */ - @Test - public void testGroupedReduceWithoutCombiner() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - DataSet> data = - env.readCsvFile("file:///will/never/be/read") - .types(String.class, Double.class) - .name("source") - .setParallelism(6); - - data.groupBy(0) - .reduce( - new RichReduceFunction>() { - @Override - public Tuple2 reduce( - Tuple2 value1, Tuple2 value2) { - return null; - } - }) - .setCombineHint(CombineHint.NONE) - .name("reducer") - .output(new DiscardingOutputFormat>()) - .name("sink"); - - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(op); - - // get the original nodes - SourcePlanNode sourceNode = resolver.getNode("source"); - SingleInputPlanNode reduceNode = resolver.getNode("reducer"); - SinkPlanNode sinkNode = resolver.getNode("sink"); - - // check wiring - assertEquals(sourceNode, reduceNode.getInput().getSource()); - - // check the strategies - assertEquals(DriverStrategy.SORTED_REDUCE, reduceNode.getDriverStrategy()); - - // check the keys - assertEquals(new FieldList(0), reduceNode.getKeys(0)); - assertEquals(new FieldList(0), reduceNode.getInput().getLocalStrategyKeys()); - - // check parallelism - assertEquals(6, sourceNode.getParallelism()); - assertEquals(8, reduceNode.getParallelism()); - assertEquals(8, sinkNode.getParallelism()); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java deleted file mode 100644 index 4a460b942cc93..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/java/WorksetIterationsJavaApiCompilerTest.java +++ /dev/null @@ -1,377 +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.optimizer.java; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.common.functions.RichJoinFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.operators.util.FieldList; -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Tests that validate optimizer choices when using operators that are requesting certain specific - * execution strategies. - */ -@SuppressWarnings("serial") -public class WorksetIterationsJavaApiCompilerTest extends CompilerTestBase { - - private static final String JOIN_WITH_INVARIANT_NAME = "Test Join Invariant"; - private static final String JOIN_WITH_SOLUTION_SET = "Test Join SolutionSet"; - private static final String NEXT_WORKSET_REDUCER_NAME = "Test Reduce Workset"; - private static final String SOLUTION_DELTA_MAPPER_NAME = "Test Map Delta"; - - @Test - public void testJavaApiWithDeferredSoltionSetUpdateWithMapper() { - try { - Plan plan = getJavaTestPlan(false, true); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); - DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); - SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); - SingleInputPlanNode deltaMapper = resolver.getNode(SOLUTION_DELTA_MAPPER_NAME); - - // iteration preserves partitioning in reducer, so the first partitioning is out of the - // loop, - // the in-loop partitioning is before the final reducer - - // verify joinWithInvariant - assertEquals( - ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithInvariantNode.getInput2().getShipStrategy()); - assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); - assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); - - // verify joinWithSolutionSet - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, - joinWithSolutionSetNode.getInput2().getShipStrategy()); - assertEquals(new FieldList(1, 0), joinWithSolutionSetNode.getKeysForInput1()); - - // verify reducer - assertEquals( - ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); - assertEquals(new FieldList(1, 2), worksetReducer.getKeys(0)); - - // currently, the system may partition before or after the mapper - ShipStrategyType ss1 = deltaMapper.getInput().getShipStrategy(); - ShipStrategyType ss2 = deltaMapper.getOutgoingChannels().get(0).getShipStrategy(); - - assertTrue( - (ss1 == ShipStrategyType.FORWARD && ss2 == ShipStrategyType.PARTITION_HASH) - || (ss2 == ShipStrategyType.FORWARD - && ss1 == ShipStrategyType.PARTITION_HASH)); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - @Test - public void testJavaApiWithDeferredSoltionSetUpdateWithNonPreservingJoin() { - try { - Plan plan = getJavaTestPlan(false, false); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); - DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); - SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); - - // iteration preserves partitioning in reducer, so the first partitioning is out of the - // loop, - // the in-loop partitioning is before the final reducer - - // verify joinWithInvariant - assertEquals( - ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithInvariantNode.getInput2().getShipStrategy()); - assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); - assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); - - // verify joinWithSolutionSet - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, - joinWithSolutionSetNode.getInput2().getShipStrategy()); - assertEquals(new FieldList(1, 0), joinWithSolutionSetNode.getKeysForInput1()); - - // verify reducer - assertEquals( - ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy()); - assertEquals(new FieldList(1, 2), worksetReducer.getKeys(0)); - - // verify solution delta - assertEquals(2, joinWithSolutionSetNode.getOutgoingChannels().size()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getOutgoingChannels().get(1).getShipStrategy()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - @Test - public void testJavaApiWithDirectSoltionSetUpdate() { - try { - Plan plan = getJavaTestPlan(true, false); - - OptimizedPlan oPlan = compileNoStats(plan); - - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan); - DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME); - DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET); - SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME); - - // iteration preserves partitioning in reducer, so the first partitioning is out of the - // loop, - // the in-loop partitioning is before the final reducer - - // verify joinWithInvariant - assertEquals( - ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithInvariantNode.getInput2().getShipStrategy()); - assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput1()); - assertEquals(new FieldList(1, 2), joinWithInvariantNode.getKeysForInput2()); - - // verify joinWithSolutionSet - assertEquals( - ShipStrategyType.PARTITION_HASH, - joinWithSolutionSetNode.getInput1().getShipStrategy()); - assertEquals( - ShipStrategyType.FORWARD, - joinWithSolutionSetNode.getInput2().getShipStrategy()); - assertEquals(new FieldList(1, 0), joinWithSolutionSetNode.getKeysForInput1()); - - // verify reducer - assertEquals(ShipStrategyType.FORWARD, worksetReducer.getInput().getShipStrategy()); - assertEquals(new FieldList(1, 2), worksetReducer.getKeys(0)); - - // verify solution delta - assertEquals(1, joinWithSolutionSetNode.getOutgoingChannels().size()); - assertEquals( - ShipStrategyType.FORWARD, - joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy()); - - new JobGraphGenerator().compileJobGraph(oPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - @Test - public void testRejectPlanIfSolutionSetKeysAndJoinKeysDontMatch() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - @SuppressWarnings("unchecked") - DataSet> solutionSetInput = - env.fromElements(new Tuple3(1L, 2L, 3L)).name("Solution Set"); - @SuppressWarnings("unchecked") - DataSet> worksetInput = - env.fromElements(new Tuple3(1L, 2L, 3L)).name("Workset"); - @SuppressWarnings("unchecked") - DataSet> invariantInput = - env.fromElements(new Tuple3(1L, 2L, 3L)) - .name("Invariant Input"); - - DeltaIteration, Tuple3> iter = - solutionSetInput.iterateDelta(worksetInput, 100, 1, 2); - - DataSet> result = - iter.getWorkset() - .join(invariantInput) - .where(1, 2) - .equalTo(1, 2) - .with( - new JoinFunction< - Tuple3, - Tuple3, - Tuple3>() { - public Tuple3 join( - Tuple3 first, - Tuple3 second) { - return first; - } - }); - - try { - result.join(iter.getSolutionSet()) - .where(1, 0) - .equalTo(0, 2) - .with( - new JoinFunction< - Tuple3, - Tuple3, - Tuple3>() { - public Tuple3 join( - Tuple3 first, - Tuple3 second) { - return second; - } - }); - fail("The join should be rejected with key type mismatches."); - } catch (InvalidProgramException e) { - // expected! - } - - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test errored: " + e.getMessage()); - } - } - - private Plan getJavaTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - @SuppressWarnings("unchecked") - DataSet> solutionSetInput = - env.fromElements(new Tuple3(1L, 2L, 3L)).name("Solution Set"); - @SuppressWarnings("unchecked") - DataSet> worksetInput = - env.fromElements(new Tuple3(1L, 2L, 3L)).name("Workset"); - @SuppressWarnings("unchecked") - DataSet> invariantInput = - env.fromElements(new Tuple3(1L, 2L, 3L)).name("Invariant Input"); - - DeltaIteration, Tuple3> iter = - solutionSetInput.iterateDelta(worksetInput, 100, 1, 2); - - DataSet> joinedWithSolutionSet = - iter.getWorkset() - .join(invariantInput) - .where(1, 2) - .equalTo(1, 2) - .with( - new RichJoinFunction< - Tuple3, - Tuple3, - Tuple3>() { - public Tuple3 join( - Tuple3 first, - Tuple3 second) { - return first; - } - }) - .name(JOIN_WITH_INVARIANT_NAME) - .join(iter.getSolutionSet()) - .where(1, 0) - .equalTo(1, 2) - .with( - new RichJoinFunction< - Tuple3, - Tuple3, - Tuple3>() { - public Tuple3 join( - Tuple3 first, - Tuple3 second) { - return second; - } - }) - .name(JOIN_WITH_SOLUTION_SET) - .withForwardedFieldsSecond( - joinPreservesSolutionSet - ? new String[] {"0->0", "1->1", "2->2"} - : null); - - DataSet> nextWorkset = - joinedWithSolutionSet - .groupBy(1, 2) - .reduceGroup( - new RichGroupReduceFunction< - Tuple3, Tuple3>() { - public void reduce( - Iterable> values, - Collector> out) {} - }) - .name(NEXT_WORKSET_REDUCER_NAME) - .withForwardedFields("1->1", "2->2", "0->0"); - - DataSet> nextSolutionSet = - mapBeforeSolutionDelta - ? joinedWithSolutionSet - .map( - new RichMapFunction< - Tuple3, - Tuple3>() { - public Tuple3 map( - Tuple3 value) { - return value; - } - }) - .name(SOLUTION_DELTA_MAPPER_NAME) - .withForwardedFields("0->0", "1->1", "2->2") - : joinedWithSolutionSet; - - iter.closeWith(nextSolutionSet, nextWorkset) - .output(new DiscardingOutputFormat>()); - - return env.createProgramPlan(); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupGlobalPropertiesCompatibilityTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupGlobalPropertiesCompatibilityTest.java deleted file mode 100644 index 28af46c3948f3..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupGlobalPropertiesCompatibilityTest.java +++ /dev/null @@ -1,288 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class CoGroupGlobalPropertiesCompatibilityTest { - - @Test - public void checkCompatiblePartitionings() { - try { - final FieldList keysLeft = new FieldList(1, 4); - final FieldList keysRight = new FieldList(3, 1); - - CoGroupDescriptor descr = new CoGroupDescriptor(keysLeft, keysRight); - - // test compatible hash partitioning - { - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setHashPartitioned(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setHashPartitioned(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setHashPartitioned(keysLeft); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setHashPartitioned(keysRight); - - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test compatible custom partitioning - { - Partitioner part = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setCustomPartitioned(keysLeft, part); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setCustomPartitioned(keysRight, part); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setCustomPartitioned(keysLeft, part); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part); - - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test custom partitioning matching any partitioning - { - Partitioner part = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setAnyPartitioning(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setAnyPartitioning(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setCustomPartitioned(keysLeft, part); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part); - - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - TestDistribution dist1 = new TestDistribution(1); - TestDistribution dist2 = new TestDistribution(1); - - // test compatible range partitioning with one ordering - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering2.appendOrdering(field, null, Order.ASCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - // test compatible range partitioning with two orderings - { - Ordering ordering1 = new Ordering(); - ordering1.appendOrdering(keysLeft.get(0), null, Order.DESCENDING); - ordering1.appendOrdering(keysLeft.get(1), null, Order.ASCENDING); - Ordering ordering2 = new Ordering(); - ordering2.appendOrdering(keysRight.get(0), null, Order.DESCENDING); - ordering2.appendOrdering(keysRight.get(1), null, Order.ASCENDING); - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void checkInompatiblePartitionings() { - try { - final FieldList keysLeft = new FieldList(1); - final FieldList keysRight = new FieldList(3); - - final Partitioner part = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - final Partitioner part2 = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - - CoGroupDescriptor descr = new CoGroupDescriptor(keysLeft, keysRight); - - // test incompatible hash with custom partitioning - { - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setAnyPartitioning(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setAnyPartitioning(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setHashPartitioned(keysLeft); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part); - - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test incompatible custom partitionings - { - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setAnyPartitioning(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setAnyPartitioning(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setCustomPartitioned(keysLeft, part); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part2); - - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - TestDistribution dist1 = new TestDistribution(1); - TestDistribution dist2 = new TestDistribution(1); - - // test incompatible range partitioning with different key size - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - ordering2.appendOrdering(field, null, Order.ASCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test incompatible range partitioning with different ordering - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering2.appendOrdering(field, null, Order.DESCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - TestDistribution dist3 = new TestDistribution(1); - TestDistribution dist4 = new TestDistribution(2); - - // test incompatible range partitioning with different distribution - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering2.appendOrdering(field, null, Order.ASCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist3); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist4); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist3); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist4); - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java deleted file mode 100644 index def3ed5074e4f..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupOnConflictingPartitioningsTest.java +++ /dev/null @@ -1,75 +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.optimizer.operators; - -import org.apache.flink.api.common.Plan; -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.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.testfunctions.DummyCoGroupFunction; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class CoGroupOnConflictingPartitioningsTest extends CompilerTestBase { - - @Test - public void testRejectCoGroupOnHashAndRangePartitioning() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = env.fromElements(new Tuple2(0L, 0L)); - - Configuration cfg = new Configuration(); - cfg.setString( - Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, - Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH); - cfg.setString( - Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, - Optimizer.HINT_SHIP_STRATEGY_REPARTITION_RANGE); - - input.coGroup(input) - .where(0) - .equalTo(0) - .with(new DummyCoGroupFunction, Tuple2>()) - .withParameters(cfg) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()); - - Plan p = env.createProgramPlan(); - try { - compileNoStats(p); - fail("This should fail with an exception"); - } catch (CompilerException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupWithDistributionTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupWithDistributionTest.java deleted file mode 100644 index 181f88f3f363a..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/CoGroupWithDistributionTest.java +++ /dev/null @@ -1,112 +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.optimizer.operators; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.CoGroupFunction; -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.Tuple3; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.optimizer.plan.*; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; - -import org.junit.Test; - -import static org.junit.Assert.*; - -public class CoGroupWithDistributionTest extends CompilerTestBase { - - @Test - public void CoGroupWithSameDistributionTest() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - TestDistribution testDistribution1 = new TestDistribution(3); - TestDistribution testDistribution2 = new TestDistribution(3); - - DataSet> coGrouped = - DataSetUtils.partitionByRange(set1, testDistribution1, 0) - .coGroup(DataSetUtils.partitionByRange(set2, testDistribution2, 0)) - .where(0) - .equalTo(0) - .with(new CoGroupFunc()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - Channel input1 = coGroup.getInput1(); - Channel input2 = coGroup.getInput2(); - assertEquals(ShipStrategyType.FORWARD, input1.getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, input2.getShipStrategy()); - } - - @Test - public void CoGroupWithDifferentDistributionTest() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - TestDistribution testDistribution1 = new TestDistribution(3); - TestDistribution testDistribution2 = new TestDistribution(2); - - DataSet> coGrouped = - DataSetUtils.partitionByRange(set1, testDistribution1, 0) - .coGroup(DataSetUtils.partitionByRange(set2, testDistribution2, 0)) - .where(0) - .equalTo(0) - .with(new CoGroupFunc()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode coGroup = (DualInputPlanNode) sink.getInput().getSource(); - Channel input1 = coGroup.getInput1(); - Channel input2 = coGroup.getInput2(); - assertEquals(ShipStrategyType.PARTITION_HASH, input1.getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, input2.getShipStrategy()); - } - - public static class CoGroupFunc - implements CoGroupFunction< - Tuple3, - Tuple3, - Tuple3> { - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception {} - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java deleted file mode 100644 index 08cc9aaa60926..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinGlobalPropertiesCompatibilityTest.java +++ /dev/null @@ -1,289 +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.optimizer.operators; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.common.operators.Ordering; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.optimizer.dataproperties.GlobalProperties; -import org.apache.flink.optimizer.dataproperties.RequestedGlobalProperties; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings("serial") -public class JoinGlobalPropertiesCompatibilityTest { - - @Test - public void checkCompatiblePartitionings() { - try { - final FieldList keysLeft = new FieldList(1, 4); - final FieldList keysRight = new FieldList(3, 1); - - SortMergeInnerJoinDescriptor descr = - new SortMergeInnerJoinDescriptor(keysLeft, keysRight); - - // test compatible hash partitioning - { - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setHashPartitioned(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setHashPartitioned(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setHashPartitioned(keysLeft); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setHashPartitioned(keysRight); - - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test compatible custom partitioning - { - Partitioner part = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setCustomPartitioned(keysLeft, part); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setCustomPartitioned(keysRight, part); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setCustomPartitioned(keysLeft, part); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part); - - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test custom partitioning matching any partitioning - { - Partitioner part = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setAnyPartitioning(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setAnyPartitioning(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setCustomPartitioned(keysLeft, part); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part); - - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - TestDistribution dist1 = new TestDistribution(1); - TestDistribution dist2 = new TestDistribution(1); - // test compatible range partitioning with one ordering - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering2.appendOrdering(field, null, Order.ASCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - // test compatible range partitioning with two orderings - { - Ordering ordering1 = new Ordering(); - ordering1.appendOrdering(keysLeft.get(0), null, Order.DESCENDING); - ordering1.appendOrdering(keysLeft.get(1), null, Order.ASCENDING); - Ordering ordering2 = new Ordering(); - ordering2.appendOrdering(keysRight.get(0), null, Order.DESCENDING); - ordering2.appendOrdering(keysRight.get(1), null, Order.ASCENDING); - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertTrue(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void checkInompatiblePartitionings() { - try { - final FieldList keysLeft = new FieldList(1); - final FieldList keysRight = new FieldList(3); - - final Partitioner part = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - final Partitioner part2 = - new Partitioner() { - @Override - public int partition(Object key, int numPartitions) { - return 0; - } - }; - - SortMergeInnerJoinDescriptor descr = - new SortMergeInnerJoinDescriptor(keysLeft, keysRight); - - // test incompatible hash with custom partitioning - { - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setAnyPartitioning(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setAnyPartitioning(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setHashPartitioned(keysLeft); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part); - - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test incompatible custom partitionings - { - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setAnyPartitioning(keysLeft); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setAnyPartitioning(keysRight); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setCustomPartitioned(keysLeft, part); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setCustomPartitioned(keysRight, part2); - - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - TestDistribution dist1 = new TestDistribution(1); - TestDistribution dist2 = new TestDistribution(1); - - // test incompatible range partitioning with different key size - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - ordering2.appendOrdering(field, null, Order.ASCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - // test incompatible range partitioning with different ordering - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering2.appendOrdering(field, null, Order.DESCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist1); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist2); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist1); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist2); - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - - TestDistribution dist3 = new TestDistribution(1); - TestDistribution dist4 = new TestDistribution(2); - - // test incompatible range partitioning with different distribution - { - Ordering ordering1 = new Ordering(); - for (int field : keysLeft) { - ordering1.appendOrdering(field, null, Order.ASCENDING); - } - Ordering ordering2 = new Ordering(); - for (int field : keysRight) { - ordering2.appendOrdering(field, null, Order.ASCENDING); - } - - RequestedGlobalProperties reqLeft = new RequestedGlobalProperties(); - reqLeft.setRangePartitioned(ordering1, dist3); - RequestedGlobalProperties reqRight = new RequestedGlobalProperties(); - reqRight.setRangePartitioned(ordering2, dist4); - - GlobalProperties propsLeft = new GlobalProperties(); - propsLeft.setRangePartitioned(ordering1, dist3); - GlobalProperties propsRight = new GlobalProperties(); - propsRight.setRangePartitioned(ordering2, dist4); - assertFalse(descr.areCompatible(reqLeft, reqRight, propsLeft, propsRight)); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java deleted file mode 100644 index 51d391426e67d..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinOnConflictingPartitioningsTest.java +++ /dev/null @@ -1,73 +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.optimizer.operators; - -import org.apache.flink.api.common.Plan; -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.configuration.Configuration; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.junit.Test; - -import static org.junit.Assert.*; - -@SuppressWarnings({"serial", "unchecked"}) -public class JoinOnConflictingPartitioningsTest extends CompilerTestBase { - - @Test - public void testRejectJoinOnHashAndRangePartitioning() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = env.fromElements(new Tuple2(0L, 0L)); - - Configuration cfg = new Configuration(); - cfg.setString( - Optimizer.HINT_SHIP_STRATEGY_FIRST_INPUT, - Optimizer.HINT_SHIP_STRATEGY_REPARTITION_HASH); - cfg.setString( - Optimizer.HINT_SHIP_STRATEGY_SECOND_INPUT, - Optimizer.HINT_SHIP_STRATEGY_REPARTITION_RANGE); - - input.join(input) - .where(0) - .equalTo(0) - .withParameters(cfg) - .output( - new DiscardingOutputFormat< - Tuple2, Tuple2>>()); - - Plan p = env.createProgramPlan(); - try { - compileNoStats(p); - fail("This should fail with an exception"); - } catch (CompilerException e) { - // expected - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinWithDistributionTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinWithDistributionTest.java deleted file mode 100644 index 6fba6f7d52b3b..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/JoinWithDistributionTest.java +++ /dev/null @@ -1,111 +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.optimizer.operators; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.JoinFunction; -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.Tuple3; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.optimizer.plan.*; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class JoinWithDistributionTest extends CompilerTestBase { - - @Test - public void JoinWithSameDistributionTest() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - TestDistribution dist1 = new TestDistribution(3); - TestDistribution dist2 = new TestDistribution(3); - - DataSet> coGrouped = - DataSetUtils.partitionByRange(set1, dist1, 0) - .join(DataSetUtils.partitionByRange(set2, dist2, 0)) - .where(0) - .equalTo(0) - .with(new JoinFunc()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - Channel input1 = join.getInput1(); - Channel input2 = join.getInput2(); - assertEquals(ShipStrategyType.FORWARD, input1.getShipStrategy()); - assertEquals(ShipStrategyType.FORWARD, input2.getShipStrategy()); - } - - @Test - public void JoinWithDifferentDistributionTest() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> set1 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - DataSet> set2 = - env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class); - - TestDistribution dist1 = new TestDistribution(3); - TestDistribution dist2 = new TestDistribution(4); - - DataSet> coGrouped = - DataSetUtils.partitionByRange(set1, dist1, 0) - .join(DataSetUtils.partitionByRange(set2, dist2, 0)) - .where(0) - .equalTo(0) - .with(new JoinFunc()); - - coGrouped.output(new DiscardingOutputFormat>()); - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = compileWithStats(plan); - - SinkPlanNode sink = oPlan.getDataSinks().iterator().next(); - DualInputPlanNode join = (DualInputPlanNode) sink.getInput().getSource(); - Channel input1 = join.getInput1(); - Channel input2 = join.getInput2(); - assertEquals(ShipStrategyType.PARTITION_HASH, input1.getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, input2.getShipStrategy()); - } - - public static class JoinFunc - implements JoinFunction< - Tuple3, - Tuple3, - Tuple3> { - - @Override - public Tuple3 join( - Tuple3 first, Tuple3 second) - throws Exception { - return null; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/TestDistribution.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/TestDistribution.java deleted file mode 100644 index e9cdea924c243..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/operators/TestDistribution.java +++ /dev/null @@ -1,67 +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.optimizer.operators; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; - -public class TestDistribution implements DataDistribution { - - public int boundary; - - public TestDistribution(int boundary) { - this.boundary = boundary; - } - - @Override - public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - return new Object[0]; - } - - @Override - public int getNumberOfFields() { - return 1; - } - - @Override - public TypeInformation[] getKeyTypes() { - return new TypeInformation[] {BasicTypeInfo.INT_TYPE_INFO}; - } - - @Override - public void write(DataOutputView out) throws IOException {} - - @Override - public void read(DataInputView in) throws IOException {} - - @Override - public boolean equals(Object obj) { - boolean isEqual = true; - TestDistribution dist = (TestDistribution) obj; - if (this.boundary != dist.boundary) { - isEqual = false; - } - return isEqual; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plan/ChannelTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plan/ChannelTest.java deleted file mode 100644 index e37ac5fb7e5a3..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plan/ChannelTest.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.optimizer.plan; - -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.OperatorInformation; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.dag.DataSourceNode; - -import org.junit.Assert; -import org.junit.Test; - -public class ChannelTest { - - @Test - public void testGetEstimatesNoReplicationFactor() { - final long NUM_RECORD = 1001; - final long SIZE = 467131; - - DataSourceNode source = getSourceNode(); - SourcePlanNode planNode = new SourcePlanNode(source, "test node"); - Channel channel = new Channel(planNode); - - // no estimates here - Assert.assertEquals(-1, channel.getEstimatedOutputSize()); - Assert.assertEquals(-1, channel.getEstimatedNumRecords()); - - // set estimates - source.setEstimatedNumRecords(NUM_RECORD); - source.setEstimatedOutputSize(SIZE); - Assert.assertEquals(SIZE, channel.getEstimatedOutputSize()); - Assert.assertEquals(NUM_RECORD, channel.getEstimatedNumRecords()); - } - - @Test - public void testGetEstimatesWithReplicationFactor() { - final long NUM_RECORD = 1001; - final long SIZE = 467131; - - final int REPLICATION = 23; - - DataSourceNode source = getSourceNode(); - SourcePlanNode planNode = new SourcePlanNode(source, "test node"); - Channel channel = new Channel(planNode); - channel.setReplicationFactor(REPLICATION); - - // no estimates here - Assert.assertEquals(-1, channel.getEstimatedOutputSize()); - Assert.assertEquals(-1, channel.getEstimatedNumRecords()); - - // set estimates - source.setEstimatedNumRecords(NUM_RECORD); - source.setEstimatedOutputSize(SIZE); - Assert.assertEquals(SIZE * REPLICATION, channel.getEstimatedOutputSize()); - Assert.assertEquals(NUM_RECORD * REPLICATION, channel.getEstimatedNumRecords()); - } - - // private static final OptimizerNode getSingleInputNode() { - // return new MapNode(new MapOperatorBase>( - // new IdentityMapper(), - // new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, - // BasicTypeInfo.STRING_TYPE_INFO), - // "map")); - // } - - private static final DataSourceNode getSourceNode() { - return new DataSourceNode( - new GenericDataSourceBase( - new TextInputFormat(new Path("/ignored")), - new OperatorInformation(BasicTypeInfo.STRING_TYPE_INFO), - "source")); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plandump/ExecutionPlanUtilTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plandump/ExecutionPlanUtilTest.java deleted file mode 100644 index 22986a28494c1..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plandump/ExecutionPlanUtilTest.java +++ /dev/null @@ -1,61 +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.optimizer.plandump; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.ExecutionPlanUtil; -import org.apache.flink.api.java.tuple.Tuple2; - -import org.junit.Test; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.StringContains.containsString; - -/** - * Tests for {@link ExecutionPlanUtil}. We have to test this here in flink-optimizer because the - * util only works when {@link ExecutionPlanJSONGenerator} is available, which is in - * flink-optimizer. - */ -public class ExecutionPlanUtilTest { - - @Test - public void executionPlanCanBeRetrieved() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(8); - - env.readCsvFile("file:///will/never/be/executed") - .types(String.class, Double.class) - .name("sourceThatWillNotRun") - .map((in) -> in) - .returns(new TypeHint>() {}) - .name("theMap") - .writeAsText("file:///will/not/be/executed") - .name("sinkThatWillNotRun"); - - Plan plan = env.createProgramPlan(); - String executionPlanAsJSON = ExecutionPlanUtil.getExecutionPlanAsJSON(plan); - - assertThat(executionPlanAsJSON, containsString("sourceThatWillNotRun")); - assertThat(executionPlanAsJSON, containsString("sinkThatWillNotRun")); - assertThat(executionPlanAsJSON, containsString("theMap")); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plandump/NumberFormattingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plandump/NumberFormattingTest.java deleted file mode 100644 index b5a54b2b94379..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plandump/NumberFormattingTest.java +++ /dev/null @@ -1,50 +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.optimizer.plandump; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class NumberFormattingTest { - - @Test - public void testFormatNumberNoDigit() { - assertEquals("0.0", PlanJSONDumpGenerator.formatNumber(0)); - assertEquals("0.00", PlanJSONDumpGenerator.formatNumber(0.0000000001)); - assertEquals("-1.0", PlanJSONDumpGenerator.formatNumber(-1.0)); - assertEquals("1.00", PlanJSONDumpGenerator.formatNumber(1)); - assertEquals("17.00", PlanJSONDumpGenerator.formatNumber(17)); - assertEquals("17.44", PlanJSONDumpGenerator.formatNumber(17.44)); - assertEquals("143.00", PlanJSONDumpGenerator.formatNumber(143)); - assertEquals("143.40", PlanJSONDumpGenerator.formatNumber(143.4)); - assertEquals("143.50", PlanJSONDumpGenerator.formatNumber(143.5)); - assertEquals("143.60", PlanJSONDumpGenerator.formatNumber(143.6)); - assertEquals("143.45", PlanJSONDumpGenerator.formatNumber(143.45)); - assertEquals("143.55", PlanJSONDumpGenerator.formatNumber(143.55)); - assertEquals("143.65", PlanJSONDumpGenerator.formatNumber(143.65)); - assertEquals("143.66", PlanJSONDumpGenerator.formatNumber(143.655)); - - assertEquals("1.13 K", PlanJSONDumpGenerator.formatNumber(1126.0)); - assertEquals("11.13 K", PlanJSONDumpGenerator.formatNumber(11126.0)); - assertEquals("118.13 K", PlanJSONDumpGenerator.formatNumber(118126.0)); - - assertEquals("1.44 M", PlanJSONDumpGenerator.formatNumber(1435126.0)); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java deleted file mode 100644 index 1fb5d580ea930..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java +++ /dev/null @@ -1,359 +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.optimizer.plantranslate; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.ResourceSpec; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.BlockingShuffleOutputFormat; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DataSink; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.operators.Operator; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.testfunctions.IdentityMapper; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphUtils; -import org.apache.flink.runtime.jobgraph.JobType; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.util.AbstractID; - -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.lang.reflect.Method; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.Map; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -public class JobGraphGeneratorTest { - - @Rule public final TemporaryFolder tmp = new TemporaryFolder(); - - /** - * Verifies that the resources are merged correctly for chained operators when generating job - * graph - */ - @Test - public void testResourcesForChainedOperators() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder(0.1, 100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder(0.2, 200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder(0.3, 300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder(0.4, 400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder(0.5, 500).build(); - ResourceSpec resource6 = ResourceSpec.newBuilder(0.6, 600).build(); - ResourceSpec resource7 = ResourceSpec.newBuilder(0.7, 700).build(); - - Method opMethod = Operator.class.getDeclaredMethod("setResources", ResourceSpec.class); - opMethod.setAccessible(true); - - Method sinkMethod = DataSink.class.getDeclaredMethod("setResources", ResourceSpec.class); - sinkMethod.setAccessible(true); - - MapFunction mapFunction = - new MapFunction() { - @Override - public Long map(Long value) throws Exception { - return value; - } - }; - - FilterFunction filterFunction = - new FilterFunction() { - @Override - public boolean filter(Long value) throws Exception { - return false; - } - }; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input = env.fromElements(1L, 2L, 3L); - opMethod.invoke(input, resource1); - - DataSet map1 = input.map(mapFunction); - opMethod.invoke(map1, resource2); - - // CHAIN(Source -> Map -> Filter) - DataSet filter1 = map1.filter(filterFunction); - opMethod.invoke(filter1, resource3); - - IterativeDataSet startOfIteration = filter1.iterate(10); - opMethod.invoke(startOfIteration, resource4); - - DataSet map2 = startOfIteration.map(mapFunction); - opMethod.invoke(map2, resource5); - - // CHAIN(Map -> Filter) - DataSet feedback = map2.filter(filterFunction); - opMethod.invoke(feedback, resource6); - - DataSink sink = - startOfIteration.closeWith(feedback).output(new DiscardingOutputFormat()); - sinkMethod.invoke(sink, resource7); - - JobGraph jobGraph = compileJob(env); - - JobVertex sourceMapFilterVertex = - jobGraph.getVerticesSortedTopologicallyFromSources().get(0); - JobVertex iterationHeadVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(1); - JobVertex feedbackVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(2); - JobVertex sinkVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(3); - JobVertex iterationSyncVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(4); - - assertTrue( - sourceMapFilterVertex - .getMinResources() - .equals(resource1.merge(resource2).merge(resource3))); - assertTrue(iterationHeadVertex.getPreferredResources().equals(resource4)); - assertTrue(feedbackVertex.getMinResources().equals(resource5.merge(resource6))); - assertTrue(sinkVertex.getPreferredResources().equals(resource7)); - assertTrue(iterationSyncVertex.getMinResources().equals(resource4)); - } - - /** - * Verifies that the resources are set onto each job vertex correctly when generating job graph - * which covers the delta iteration case - */ - @Test - public void testResourcesForDeltaIteration() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder(0.1, 100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder(0.2, 200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder(0.3, 300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder(0.4, 400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder(0.5, 500).build(); - ResourceSpec resource6 = ResourceSpec.newBuilder(0.6, 600).build(); - - Method opMethod = Operator.class.getDeclaredMethod("setResources", ResourceSpec.class); - opMethod.setAccessible(true); - - Method deltaMethod = - DeltaIteration.class.getDeclaredMethod("setResources", ResourceSpec.class); - deltaMethod.setAccessible(true); - - Method sinkMethod = DataSink.class.getDeclaredMethod("setResources", ResourceSpec.class); - sinkMethod.setAccessible(true); - - MapFunction, Tuple2> mapFunction = - new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple2 value) throws Exception { - return value; - } - }; - - FilterFunction> filterFunction = - new FilterFunction>() { - @Override - public boolean filter(Tuple2 value) throws Exception { - return false; - } - }; - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = env.fromElements(new Tuple2<>(1L, 2L)); - opMethod.invoke(input, resource1); - - // CHAIN(Map -> Filter) - DataSet> map = input.map(mapFunction); - opMethod.invoke(map, resource2); - - DeltaIteration, Tuple2> iteration = - map.iterateDelta(map, 100, 0).registerAggregator("test", new LongSumAggregator()); - deltaMethod.invoke(iteration, resource3); - - DataSet> delta = iteration.getWorkset().map(mapFunction); - opMethod.invoke(delta, resource4); - - DataSet> feedback = delta.filter(filterFunction); - opMethod.invoke(feedback, resource5); - - DataSink> sink = - iteration - .closeWith(delta, feedback) - .output(new DiscardingOutputFormat>()); - sinkMethod.invoke(sink, resource6); - - JobGraph jobGraph = compileJob(env); - - JobVertex sourceMapVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(0); - JobVertex iterationHeadVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(1); - JobVertex deltaVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(2); - JobVertex iterationTailVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(3); - JobVertex feedbackVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(4); - JobVertex sinkVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(5); - JobVertex iterationSyncVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(6); - - assertTrue(sourceMapVertex.getMinResources().equals(resource1.merge(resource2))); - assertTrue(iterationHeadVertex.getPreferredResources().equals(resource3)); - assertTrue(deltaVertex.getMinResources().equals(resource4)); - // the iteration tail task will be scheduled in the same instance with iteration head, and - // currently not set resources. - assertTrue(iterationTailVertex.getPreferredResources().equals(ResourceSpec.DEFAULT)); - assertTrue(feedbackVertex.getMinResources().equals(resource5)); - assertTrue(sinkVertex.getPreferredResources().equals(resource6)); - assertTrue(iterationSyncVertex.getMinResources().equals(resource3)); - } - - @Test - public void testArtifactCompression() throws IOException { - Path plainFile1 = tmp.newFile("plainFile1").toPath(); - Path plainFile2 = tmp.newFile("plainFile2").toPath(); - - Path directory1 = tmp.newFolder("directory1").toPath(); - Files.createDirectory(directory1.resolve("containedFile1")); - - Path directory2 = tmp.newFolder("directory2").toPath(); - Files.createDirectory(directory2.resolve("containedFile2")); - - final String executableFileName = "executableFile"; - final String nonExecutableFileName = "nonExecutableFile"; - final String executableDirName = "executableDir"; - final String nonExecutableDirName = "nonExecutableDIr"; - - Map originalArtifacts = new HashMap<>(); - originalArtifacts.put( - executableFileName, - new DistributedCache.DistributedCacheEntry(plainFile1.toString(), true)); - originalArtifacts.put( - nonExecutableFileName, - new DistributedCache.DistributedCacheEntry(plainFile2.toString(), false)); - originalArtifacts.put( - executableDirName, - new DistributedCache.DistributedCacheEntry(directory1.toString(), true)); - originalArtifacts.put( - nonExecutableDirName, - new DistributedCache.DistributedCacheEntry(directory2.toString(), false)); - - final Map submittedArtifacts = - JobGraphUtils.prepareUserArtifactEntries(originalArtifacts, new JobID()); - - DistributedCache.DistributedCacheEntry executableFileEntry = - submittedArtifacts.get(executableFileName); - assertState(executableFileEntry, true, false); - - DistributedCache.DistributedCacheEntry nonExecutableFileEntry = - submittedArtifacts.get(nonExecutableFileName); - assertState(nonExecutableFileEntry, false, false); - - DistributedCache.DistributedCacheEntry executableDirEntry = - submittedArtifacts.get(executableDirName); - assertState(executableDirEntry, true, true); - - DistributedCache.DistributedCacheEntry nonExecutableDirEntry = - submittedArtifacts.get(nonExecutableDirName); - assertState(nonExecutableDirEntry, false, true); - } - - @Test - public void testGeneratedJobsAreBatchJobType() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements("test").output(new DiscardingOutputFormat<>()); - - JobGraph graph = compileJob(env); - assertThat(graph.getJobType(), is(JobType.BATCH)); - } - - @Test - public void testGeneratingJobGraphWithUnconsumedResultPartition() { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = - env.fromElements(new Tuple2<>(1L, 2L)).setParallelism(1); - - DataSet> ds = input.map(new IdentityMapper<>()).setParallelism(3); - - AbstractID intermediateDataSetID = new AbstractID(); - - // this output branch will be excluded. - ds.output(BlockingShuffleOutputFormat.createOutputFormat(intermediateDataSetID)) - .setParallelism(1); - - // this is the normal output branch. - ds.output(new DiscardingOutputFormat<>()).setParallelism(1); - - JobGraph jobGraph = compileJob(env); - - Assert.assertEquals(3, jobGraph.getVerticesSortedTopologicallyFromSources().size()); - - JobVertex mapVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(1); - Assert.assertThat(mapVertex, Matchers.instanceOf(JobVertex.class)); - - // there are 2 output result with one of them is ResultPartitionType.BLOCKING_PERSISTENT - Assert.assertEquals(2, mapVertex.getProducedDataSets().size()); - - Assert.assertTrue( - mapVertex.getProducedDataSets().stream() - .anyMatch( - dataSet -> - dataSet.getId() - .equals( - new IntermediateDataSetID( - intermediateDataSetID)) - && dataSet.getResultType() - == ResultPartitionType - .BLOCKING_PERSISTENT)); - } - - private static void assertState( - DistributedCache.DistributedCacheEntry entry, boolean isExecutable, boolean isZipped) - throws IOException { - assertNotNull(entry); - assertEquals(isExecutable, entry.isExecutable); - assertEquals(isZipped, entry.isZipped); - org.apache.flink.core.fs.Path filePath = new org.apache.flink.core.fs.Path(entry.filePath); - assertTrue(filePath.getFileSystem().exists(filePath)); - assertFalse(filePath.getFileSystem().getFileStatus(filePath).isDir()); - } - - private static JobGraph compileJob(ExecutionEnvironment env) { - Plan plan = env.createProgramPlan(); - Optimizer pc = new Optimizer(new Configuration()); - OptimizedPlan op = pc.compile(plan); - - JobGraphGenerator jgg = new JobGraphGenerator(); - return jgg.compileJobGraph(op); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/TempInIterationsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/TempInIterationsTest.java deleted file mode 100644 index 7c7b0c4665777..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/TempInIterationsTest.java +++ /dev/null @@ -1,85 +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.optimizer.plantranslate; - -import org.apache.flink.api.common.Plan; -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.testfunctions.DummyFlatJoinFunction; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.operators.util.TaskConfig; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; - -public class TempInIterationsTest { - - /* - * Tests whether temps barriers are correctly set in within iterations - */ - @Test - public void testTempInIterationTest() throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = - env.readCsvFile("file:///does/not/exist").types(Long.class, Long.class); - - DeltaIteration, Tuple2> iteration = - input.iterateDelta(input, 1, 0); - - DataSet> update = - iteration - .getWorkset() - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new DummyFlatJoinFunction>()); - - iteration - .closeWith(update, update) - .output(new DiscardingOutputFormat>()); - - Plan plan = env.createProgramPlan(); - OptimizedPlan oPlan = (new Optimizer(new Configuration())).compile(plan); - - JobGraphGenerator jgg = new JobGraphGenerator(); - JobGraph jg = jgg.compileJobGraph(oPlan); - - boolean solutionSetUpdateChecked = false; - for (JobVertex v : jg.getVertices()) { - if (v.getName().equals("SolutionSet Delta")) { - - // check if input of solution set delta is temped - TaskConfig tc = new TaskConfig(v.getConfiguration()); - assertTrue(tc.isInputAsynchronouslyMaterialized(0)); - solutionSetUpdateChecked = true; - } - } - assertTrue(solutionSetUpdateChecked); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/programs/ConnectedComponentsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/programs/ConnectedComponentsTest.java deleted file mode 100644 index 45dd599d9a327..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/programs/ConnectedComponentsTest.java +++ /dev/null @@ -1,357 +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.optimizer.programs; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.util.FieldList; -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.io.network.DataExchangeMode; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; - -@SuppressWarnings("serial") -public class ConnectedComponentsTest extends CompilerTestBase { - - private static final String VERTEX_SOURCE = "Vertices"; - - private static final String ITERATION_NAME = "Connected Components Iteration"; - - private static final String EDGES_SOURCE = "Edges"; - private static final String JOIN_NEIGHBORS_MATCH = "Join Candidate Id With Neighbor"; - private static final String MIN_ID_REDUCER = "Find Minimum Candidate Id"; - private static final String UPDATE_ID_MATCH = "Update Component Id"; - - private static final String SINK = "Result"; - - private final FieldList set0 = new FieldList(0); - - @Test - public void testWorksetConnectedComponents() { - Plan plan = getConnectedComponentsPlan(DEFAULT_PARALLELISM, 100, false); - - OptimizedPlan optPlan = compileNoStats(plan); - OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan); - - SourcePlanNode vertexSource = or.getNode(VERTEX_SOURCE); - SourcePlanNode edgesSource = or.getNode(EDGES_SOURCE); - SinkPlanNode sink = or.getNode(SINK); - WorksetIterationPlanNode iter = or.getNode(ITERATION_NAME); - - DualInputPlanNode neighborsJoin = or.getNode(JOIN_NEIGHBORS_MATCH); - SingleInputPlanNode minIdReducer = or.getNode(MIN_ID_REDUCER); - SingleInputPlanNode minIdCombiner = (SingleInputPlanNode) minIdReducer.getPredecessor(); - DualInputPlanNode updatingMatch = or.getNode(UPDATE_ID_MATCH); - - // test all drivers - Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, vertexSource.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, edgesSource.getDriverStrategy()); - - Assert.assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED, neighborsJoin.getDriverStrategy()); - Assert.assertTrue(!neighborsJoin.getInput1().getTempMode().isCached()); - Assert.assertTrue(!neighborsJoin.getInput2().getTempMode().isCached()); - Assert.assertEquals(set0, neighborsJoin.getKeysForInput1()); - Assert.assertEquals(set0, neighborsJoin.getKeysForInput2()); - - Assert.assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_SECOND, updatingMatch.getDriverStrategy()); - Assert.assertEquals(set0, updatingMatch.getKeysForInput1()); - Assert.assertEquals(set0, updatingMatch.getKeysForInput2()); - - // test all the shipping strategies - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, - iter.getInitialSolutionSetInput().getShipStrategy()); - Assert.assertEquals(set0, iter.getInitialSolutionSetInput().getShipStrategyKeys()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, iter.getInitialWorksetInput().getShipStrategy()); - Assert.assertEquals(set0, iter.getInitialWorksetInput().getShipStrategyKeys()); - - Assert.assertEquals( - ShipStrategyType.FORWARD, neighborsJoin.getInput1().getShipStrategy()); // workset - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, - neighborsJoin.getInput2().getShipStrategy()); // edges - Assert.assertEquals(set0, neighborsJoin.getInput2().getShipStrategyKeys()); - - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, minIdReducer.getInput().getShipStrategy()); - Assert.assertEquals(set0, minIdReducer.getInput().getShipStrategyKeys()); - Assert.assertEquals(ShipStrategyType.FORWARD, minIdCombiner.getInput().getShipStrategy()); - - Assert.assertEquals( - ShipStrategyType.FORWARD, updatingMatch.getInput1().getShipStrategy()); // min id - Assert.assertEquals( - ShipStrategyType.FORWARD, - updatingMatch.getInput2().getShipStrategy()); // solution set - - // test all the local strategies - Assert.assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy()); - Assert.assertEquals( - LocalStrategy.NONE, iter.getInitialSolutionSetInput().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.NONE, iter.getInitialWorksetInput().getLocalStrategy()); - - Assert.assertEquals( - LocalStrategy.NONE, neighborsJoin.getInput1().getLocalStrategy()); // workset - Assert.assertEquals( - LocalStrategy.NONE, neighborsJoin.getInput2().getLocalStrategy()); // edges - - Assert.assertEquals( - LocalStrategy.COMBININGSORT, minIdReducer.getInput().getLocalStrategy()); - Assert.assertEquals(set0, minIdReducer.getInput().getLocalStrategyKeys()); - Assert.assertEquals(LocalStrategy.NONE, minIdCombiner.getInput().getLocalStrategy()); - - Assert.assertEquals( - LocalStrategy.NONE, updatingMatch.getInput1().getLocalStrategy()); // min id - Assert.assertEquals( - LocalStrategy.NONE, updatingMatch.getInput2().getLocalStrategy()); // solution set - - // check the dams - Assert.assertEquals(TempMode.NONE, iter.getInitialWorksetInput().getTempMode()); - Assert.assertEquals(TempMode.NONE, iter.getInitialSolutionSetInput().getTempMode()); - - Assert.assertEquals( - DataExchangeMode.BATCH, iter.getInitialWorksetInput().getDataExchangeMode()); - Assert.assertEquals( - DataExchangeMode.BATCH, iter.getInitialSolutionSetInput().getDataExchangeMode()); - - JobGraphGenerator jgg = new JobGraphGenerator(); - jgg.compileJobGraph(optPlan); - } - - @Test - public void testWorksetConnectedComponentsWithSolutionSetAsFirstInput() { - - Plan plan = getConnectedComponentsPlan(DEFAULT_PARALLELISM, 100, true); - - OptimizedPlan optPlan = compileNoStats(plan); - OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan); - - SourcePlanNode vertexSource = or.getNode(VERTEX_SOURCE); - SourcePlanNode edgesSource = or.getNode(EDGES_SOURCE); - SinkPlanNode sink = or.getNode(SINK); - WorksetIterationPlanNode iter = or.getNode(ITERATION_NAME); - - DualInputPlanNode neighborsJoin = or.getNode(JOIN_NEIGHBORS_MATCH); - SingleInputPlanNode minIdReducer = or.getNode(MIN_ID_REDUCER); - SingleInputPlanNode minIdCombiner = (SingleInputPlanNode) minIdReducer.getPredecessor(); - DualInputPlanNode updatingMatch = or.getNode(UPDATE_ID_MATCH); - - // test all drivers - Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, vertexSource.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, edgesSource.getDriverStrategy()); - - Assert.assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_SECOND_CACHED, neighborsJoin.getDriverStrategy()); - Assert.assertTrue(!neighborsJoin.getInput1().getTempMode().isCached()); - Assert.assertTrue(!neighborsJoin.getInput2().getTempMode().isCached()); - Assert.assertEquals(set0, neighborsJoin.getKeysForInput1()); - Assert.assertEquals(set0, neighborsJoin.getKeysForInput2()); - - Assert.assertEquals( - DriverStrategy.HYBRIDHASH_BUILD_FIRST, updatingMatch.getDriverStrategy()); - Assert.assertEquals(set0, updatingMatch.getKeysForInput1()); - Assert.assertEquals(set0, updatingMatch.getKeysForInput2()); - - // test all the shipping strategies - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, - iter.getInitialSolutionSetInput().getShipStrategy()); - Assert.assertEquals(set0, iter.getInitialSolutionSetInput().getShipStrategyKeys()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, iter.getInitialWorksetInput().getShipStrategy()); - Assert.assertEquals(set0, iter.getInitialWorksetInput().getShipStrategyKeys()); - - Assert.assertEquals( - ShipStrategyType.FORWARD, neighborsJoin.getInput1().getShipStrategy()); // workset - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, - neighborsJoin.getInput2().getShipStrategy()); // edges - Assert.assertEquals(set0, neighborsJoin.getInput2().getShipStrategyKeys()); - - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, minIdReducer.getInput().getShipStrategy()); - Assert.assertEquals(set0, minIdReducer.getInput().getShipStrategyKeys()); - Assert.assertEquals(ShipStrategyType.FORWARD, minIdCombiner.getInput().getShipStrategy()); - - Assert.assertEquals( - ShipStrategyType.FORWARD, - updatingMatch.getInput1().getShipStrategy()); // solution set - Assert.assertEquals( - ShipStrategyType.FORWARD, updatingMatch.getInput2().getShipStrategy()); // min id - - // test all the local strategies - Assert.assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy()); - Assert.assertEquals( - LocalStrategy.NONE, iter.getInitialSolutionSetInput().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.NONE, iter.getInitialWorksetInput().getLocalStrategy()); - - Assert.assertEquals( - LocalStrategy.NONE, neighborsJoin.getInput1().getLocalStrategy()); // workset - Assert.assertEquals( - LocalStrategy.NONE, neighborsJoin.getInput2().getLocalStrategy()); // edges - - Assert.assertEquals( - LocalStrategy.COMBININGSORT, minIdReducer.getInput().getLocalStrategy()); - Assert.assertEquals(set0, minIdReducer.getInput().getLocalStrategyKeys()); - Assert.assertEquals(LocalStrategy.NONE, minIdCombiner.getInput().getLocalStrategy()); - - Assert.assertEquals( - LocalStrategy.NONE, updatingMatch.getInput1().getLocalStrategy()); // min id - Assert.assertEquals( - LocalStrategy.NONE, updatingMatch.getInput2().getLocalStrategy()); // solution set - - // check the dams - Assert.assertEquals(TempMode.NONE, iter.getInitialWorksetInput().getTempMode()); - Assert.assertEquals(TempMode.NONE, iter.getInitialSolutionSetInput().getTempMode()); - - Assert.assertEquals( - DataExchangeMode.BATCH, iter.getInitialWorksetInput().getDataExchangeMode()); - Assert.assertEquals( - DataExchangeMode.BATCH, iter.getInitialSolutionSetInput().getDataExchangeMode()); - - JobGraphGenerator jgg = new JobGraphGenerator(); - jgg.compileJobGraph(optPlan); - } - - private static Plan getConnectedComponentsPlan( - int parallelism, int iterations, boolean solutionSetFirst) { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> verticesWithId = - env.generateSequence(0, 1000) - .name("Vertices") - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) { - return new Tuple2(value, value); - } - }) - .name("Assign Vertex Ids"); - - DeltaIteration, Tuple2> iteration = - verticesWithId - .iterateDelta(verticesWithId, iterations, 0) - .name("Connected Components Iteration"); - - @SuppressWarnings("unchecked") - DataSet> edges = - env.fromElements(new Tuple2(0L, 0L)).name("Edges"); - - DataSet> minCandidateId = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .projectSecond(1) - .>projectFirst(1) - .name("Join Candidate Id With Neighbor") - .groupBy(0) - .min(1) - .name("Find Minimum Candidate Id"); - - DataSet> updateComponentId; - - if (solutionSetFirst) { - updateComponentId = - iteration - .getSolutionSet() - .join(minCandidateId) - .where(0) - .equalTo(0) - .with( - new FlatJoinFunction< - Tuple2, - Tuple2, - Tuple2>() { - @Override - public void join( - Tuple2 current, - Tuple2 candidate, - Collector> out) { - if (candidate.f1 < current.f1) { - out.collect(candidate); - } - } - }) - .withForwardedFieldsFirst("0") - .withForwardedFieldsSecond("0") - .name("Update Component Id"); - } else { - updateComponentId = - minCandidateId - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with( - new FlatJoinFunction< - Tuple2, - Tuple2, - Tuple2>() { - @Override - public void join( - Tuple2 candidate, - Tuple2 current, - Collector> out) { - if (candidate.f1 < current.f1) { - out.collect(candidate); - } - } - }) - .withForwardedFieldsFirst("0") - .withForwardedFieldsSecond("0") - .name("Update Component Id"); - } - - iteration - .closeWith(updateComponentId, updateComponentId) - .output(new DiscardingOutputFormat>()) - .name("Result"); - - return env.createProgramPlan(); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/DummyCoGroupFunction.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/DummyCoGroupFunction.java deleted file mode 100644 index 2bc5582eac0d1..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/DummyCoGroupFunction.java +++ /dev/null @@ -1,31 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.RichCoGroupFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.util.Collector; - -public class DummyCoGroupFunction extends RichCoGroupFunction> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup(Iterable first, Iterable second, Collector> out) {} -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/DummyFlatJoinFunction.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/DummyFlatJoinFunction.java deleted file mode 100644 index 3486bf6c155ef..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/DummyFlatJoinFunction.java +++ /dev/null @@ -1,32 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.RichFlatJoinFunction; -import org.apache.flink.util.Collector; - -public class DummyFlatJoinFunction extends RichFlatJoinFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void join(T first, T second, Collector out) { - out.collect(null); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityCoGrouper.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityCoGrouper.java deleted file mode 100644 index 74af023b42ca5..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityCoGrouper.java +++ /dev/null @@ -1,30 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.util.Collector; - -public class IdentityCoGrouper implements CoGroupFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup(Iterable first, Iterable second, Collector out) {} -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityCrosser.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityCrosser.java deleted file mode 100644 index af0637c0c3485..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityCrosser.java +++ /dev/null @@ -1,31 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.CrossFunction; - -public class IdentityCrosser implements CrossFunction { - - private static final long serialVersionUID = 1L; - - @Override - public T cross(T first, T second) { - return first; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityFlatMapper.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityFlatMapper.java deleted file mode 100644 index 494c65ce5a6ff..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityFlatMapper.java +++ /dev/null @@ -1,30 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.util.Collector; - -public class IdentityFlatMapper implements FlatMapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(T value, Collector out) {} -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityGroupReducer.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityGroupReducer.java deleted file mode 100644 index cf7b6494373c8..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityGroupReducer.java +++ /dev/null @@ -1,34 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.util.Collector; - -public class IdentityGroupReducer implements GroupReduceFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - for (T next : values) { - out.collect(next); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityGroupReducerCombinable.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityGroupReducerCombinable.java deleted file mode 100644 index 8f45680ec9e16..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityGroupReducerCombinable.java +++ /dev/null @@ -1,41 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.util.Collector; - -public class IdentityGroupReducerCombinable - implements GroupReduceFunction, GroupCombineFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - for (T next : values) { - out.collect(next); - } - } - - @Override - public void combine(Iterable values, Collector out) { - reduce(values, out); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityJoiner.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityJoiner.java deleted file mode 100644 index 631162db6b8e6..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityJoiner.java +++ /dev/null @@ -1,31 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.JoinFunction; - -public class IdentityJoiner implements JoinFunction { - - private static final long serialVersionUID = 1L; - - @Override - public T join(T first, T second) { - return first; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityKeyExtractor.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityKeyExtractor.java deleted file mode 100644 index 586076c8cbe4c..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityKeyExtractor.java +++ /dev/null @@ -1,31 +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.optimizer.testfunctions; - -import org.apache.flink.api.java.functions.KeySelector; - -public class IdentityKeyExtractor implements KeySelector { - - private static final long serialVersionUID = 1L; - - @Override - public T getKey(T value) { - return value; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityMapper.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityMapper.java deleted file mode 100644 index 124461a7a96fe..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityMapper.java +++ /dev/null @@ -1,31 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.RichMapFunction; - -public class IdentityMapper extends RichMapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public T map(T value) { - return value; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityPartitionerMapper.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityPartitionerMapper.java deleted file mode 100644 index 664866d2677a8..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/IdentityPartitionerMapper.java +++ /dev/null @@ -1,34 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.RichMapPartitionFunction; -import org.apache.flink.util.Collector; - -public class IdentityPartitionerMapper extends RichMapPartitionFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void mapPartition(Iterable values, Collector out) { - for (T in : values) { - out.collect(in); - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/SelectOneReducer.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/SelectOneReducer.java deleted file mode 100644 index 3cacfbd7cdb8c..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/SelectOneReducer.java +++ /dev/null @@ -1,31 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.RichReduceFunction; - -public class SelectOneReducer extends RichReduceFunction { - - private static final long serialVersionUID = 1L; - - @Override - public T reduce(T value1, T value2) throws Exception { - return value1; - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/Top1GroupReducer.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/Top1GroupReducer.java deleted file mode 100644 index b78ea331c747a..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/testfunctions/Top1GroupReducer.java +++ /dev/null @@ -1,38 +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.optimizer.testfunctions; - -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.util.Collector; - -public class Top1GroupReducer implements GroupReduceFunction, GroupCombineFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - out.collect(values.iterator().next()); - } - - @Override - public void combine(Iterable values, Collector out) { - out.collect(values.iterator().next()); - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java deleted file mode 100644 index 4efaacd4cd9aa..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/CompilerTestBase.java +++ /dev/null @@ -1,241 +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.optimizer.util; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.BulkIterationBase; -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.plan.PlanNode; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.util.OperatingSystem; -import org.apache.flink.util.TestLogger; -import org.apache.flink.util.Visitor; - -import org.junit.Before; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -/** - * Base class for Optimizer tests. Offers utility methods to trigger optimization of a program and - * to fetch the nodes in an optimizer plan that correspond the node in the program plan. - */ -public abstract class CompilerTestBase extends TestLogger implements java.io.Serializable { - - private static final long serialVersionUID = 1L; - - protected static final String IN_FILE = - OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/random"; - - protected static final String OUT_FILE = - OperatingSystem.isWindows() ? "file:/c:/" : "file:///dev/null"; - - protected static final int DEFAULT_PARALLELISM = 8; - - protected static final String DEFAULT_PARALLELISM_STRING = String.valueOf(DEFAULT_PARALLELISM); - - private static final String CACHE_KEY = "cachekey"; - - // ------------------------------------------------------------------------ - - protected transient DataStatistics dataStats; - - protected transient Optimizer withStatsCompiler; - - protected transient Optimizer noStatsCompiler; - - private transient int statCounter; - - // ------------------------------------------------------------------------ - - @Before - public void setup() { - Configuration flinkConf = new Configuration(); - this.dataStats = new DataStatistics(); - this.withStatsCompiler = - new Optimizer(this.dataStats, new DefaultCostEstimator(), flinkConf); - this.withStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); - - this.noStatsCompiler = new Optimizer(null, new DefaultCostEstimator(), flinkConf); - this.noStatsCompiler.setDefaultParallelism(DEFAULT_PARALLELISM); - } - - // ------------------------------------------------------------------------ - - public OptimizedPlan compileWithStats(Plan p) { - return this.withStatsCompiler.compile(p); - } - - public OptimizedPlan compileNoStats(Plan p) { - return this.noStatsCompiler.compile(p); - } - - public static OperatorResolver getContractResolver(Plan plan) { - return new OperatorResolver(plan); - } - - public void setSourceStatistics( - GenericDataSourceBase source, long size, float recordWidth) { - setSourceStatistics(source, new FileBaseStatistics(Long.MAX_VALUE, size, recordWidth)); - } - - public void setSourceStatistics(GenericDataSourceBase source, FileBaseStatistics stats) { - final String key = CACHE_KEY + this.statCounter++; - this.dataStats.cacheBaseStatistics(stats, key); - source.setStatisticsKey(key); - } - - public static OptimizerPlanNodeResolver getOptimizerPlanNodeResolver(OptimizedPlan plan) { - return new OptimizerPlanNodeResolver(plan); - } - - // ------------------------------------------------------------------------ - - public static final class OptimizerPlanNodeResolver { - - private final Map> map; - - public OptimizerPlanNodeResolver(OptimizedPlan p) { - HashMap> map = new HashMap>(); - - for (PlanNode n : p.getAllNodes()) { - Operator c = n.getOriginalOptimizerNode().getOperator(); - String name = c.getName(); - - ArrayList list = map.get(name); - if (list == null) { - list = new ArrayList(2); - map.put(name, list); - } - - // check whether this node is a child of a node with the same contract (aka - // combiner) - boolean shouldAdd = true; - for (Iterator iter = list.iterator(); iter.hasNext(); ) { - PlanNode in = iter.next(); - if (in.getOriginalOptimizerNode().getOperator() == c) { - // is this the child or is our node the child - if (in instanceof SingleInputPlanNode && n instanceof SingleInputPlanNode) { - SingleInputPlanNode thisNode = (SingleInputPlanNode) n; - SingleInputPlanNode otherNode = (SingleInputPlanNode) in; - - if (thisNode.getPredecessor() == otherNode) { - // other node is child, remove it - iter.remove(); - } else if (otherNode.getPredecessor() == thisNode) { - shouldAdd = false; - } - } else { - throw new RuntimeException("Unrecodnized case in test."); - } - } - } - - if (shouldAdd) { - list.add(n); - } - } - - this.map = map; - } - - @SuppressWarnings("unchecked") - public T getNode(String name) { - List nodes = this.map.get(name); - if (nodes == null || nodes.isEmpty()) { - throw new RuntimeException("No node found with the given name."); - } else if (nodes.size() != 1) { - throw new RuntimeException("Multiple nodes found with the given name."); - } else { - return (T) nodes.get(0); - } - } - - @SuppressWarnings("unchecked") - public T getNode(String name, Class stubClass) { - List nodes = this.map.get(name); - if (nodes == null || nodes.isEmpty()) { - throw new RuntimeException("No node found with the given name and stub class."); - } else { - PlanNode found = null; - for (PlanNode node : nodes) { - if (node.getClass() == stubClass) { - if (found == null) { - found = node; - } else { - throw new RuntimeException( - "Multiple nodes found with the given name and stub class."); - } - } - } - if (found == null) { - throw new RuntimeException("No node found with the given name and stub class."); - } else { - return (T) found; - } - } - } - - public List getNodes(String name) { - List nodes = this.map.get(name); - if (nodes == null || nodes.isEmpty()) { - throw new RuntimeException("No node found with the given name."); - } else { - return new ArrayList(nodes); - } - } - } - - /** Collects all DataSources of a plan to add statistics */ - public static class SourceCollectorVisitor implements Visitor> { - - protected final List> sources = - new ArrayList>(4); - - @Override - public boolean preVisit(Operator visitable) { - - if (visitable instanceof GenericDataSourceBase) { - sources.add((GenericDataSourceBase) visitable); - } else if (visitable instanceof BulkIterationBase) { - ((BulkIterationBase) visitable).getNextPartialSolution().accept(this); - } - - return true; - } - - @Override - public void postVisit(Operator visitable) {} - - public List> getSources() { - return this.sources; - } - } -} diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java deleted file mode 100644 index 97292c911754e..0000000000000 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/util/OperatorResolver.java +++ /dev/null @@ -1,123 +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.optimizer.util; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.base.BulkIterationBase; -import org.apache.flink.api.common.operators.base.DeltaIterationBase; -import org.apache.flink.util.Visitor; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** Utility to get operator instances from plans via name. */ -public class OperatorResolver implements Visitor> { - - private final Map>> map; - private Set> seen; - - public OperatorResolver(Plan p) { - this.map = new HashMap>>(); - this.seen = new HashSet>(); - - p.accept(this); - this.seen = null; - } - - @SuppressWarnings("unchecked") - public > T getNode(String name) { - List> nodes = this.map.get(name); - if (nodes == null || nodes.isEmpty()) { - throw new RuntimeException("No nodes found with the given name."); - } else if (nodes.size() != 1) { - throw new RuntimeException("Multiple nodes found with the given name."); - } else { - return (T) nodes.get(0); - } - } - - @SuppressWarnings("unchecked") - public > T getNode(String name, Class stubClass) { - List> nodes = this.map.get(name); - if (nodes == null || nodes.isEmpty()) { - throw new RuntimeException("No node found with the given name and stub class."); - } else { - Operator found = null; - for (Operator node : nodes) { - if (node.getClass() == stubClass) { - if (found == null) { - found = node; - } else { - throw new RuntimeException( - "Multiple nodes found with the given name and stub class."); - } - } - } - if (found == null) { - throw new RuntimeException("No node found with the given name and stub class."); - } else { - return (T) found; - } - } - } - - public List> getNodes(String name) { - List> nodes = this.map.get(name); - if (nodes == null || nodes.isEmpty()) { - throw new RuntimeException("No node found with the given name."); - } else { - return new ArrayList>(nodes); - } - } - - @Override - public boolean preVisit(Operator visitable) { - if (this.seen.add(visitable)) { - // add to the map - final String name = visitable.getName(); - List> list = this.map.get(name); - if (list == null) { - list = new ArrayList>(2); - this.map.put(name, list); - } - list.add(visitable); - - // recurse into bulk iterations - if (visitable instanceof BulkIterationBase) { - ((BulkIterationBase) visitable).getNextPartialSolution().accept(this); - } else if (visitable instanceof DeltaIterationBase) { - ((DeltaIterationBase) visitable).getSolutionSetDelta().accept(this); - ((DeltaIterationBase) visitable).getNextWorkset().accept(this); - } - - return true; - } else { - return false; - } - } - - @Override - public void postVisit(Operator visitable) {} -} diff --git a/flink-optimizer/src/test/resources/log4j2-test.properties b/flink-optimizer/src/test/resources/log4j2-test.properties deleted file mode 100644 index 835c2ec9a3d02..0000000000000 --- a/flink-optimizer/src/test/resources/log4j2-test.properties +++ /dev/null @@ -1,28 +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. -################################################################################ - -# Set root logger level to OFF to not flood build logs -# set manually to INFO for debugging purposes -rootLogger.level = OFF -rootLogger.appenderRef.test.ref = TestLogger - -appender.testlogger.name = TestLogger -appender.testlogger.type = CONSOLE -appender.testlogger.target = SYSTEM_ERR -appender.testlogger.layout.type = PatternLayout -appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-python/apache-flink-libraries/setup.py b/flink-python/apache-flink-libraries/setup.py index 8e7db9dfee63f..8d010149114a8 100644 --- a/flink-python/apache-flink-libraries/setup.py +++ b/flink-python/apache-flink-libraries/setup.py @@ -49,8 +49,8 @@ def find_file_path(pattern): return files[0] -in_flink_source = os.path.isfile("../../flink-java/src/main/java/org/apache/flink/api/java/" - "ExecutionEnvironment.java") +in_flink_source = os.path.isfile("../../flink-runtime/src/main/java/org/apache/flink/streaming" + "/api/environment/StreamExecutionEnvironment.java") this_directory = os.path.abspath(os.path.dirname(__file__)) pyflink_directory = os.path.join(this_directory, "pyflink") if in_flink_source: diff --git a/flink-python/docs/reference/pyflink.datastream/stream_execution_environment.rst b/flink-python/docs/reference/pyflink.datastream/stream_execution_environment.rst index 78496a1ed4c59..ea8be1925d3d8 100644 --- a/flink-python/docs/reference/pyflink.datastream/stream_execution_environment.rst +++ b/flink-python/docs/reference/pyflink.datastream/stream_execution_environment.rst @@ -74,7 +74,6 @@ access). StreamExecutionEnvironment.create_input StreamExecutionEnvironment.add_source StreamExecutionEnvironment.from_source - StreamExecutionEnvironment.read_text_file StreamExecutionEnvironment.from_collection StreamExecutionEnvironment.is_unaligned_checkpoints_enabled StreamExecutionEnvironment.is_force_unaligned_checkpoints diff --git a/flink-python/pom.xml b/flink-python/pom.xml index 1f96abd752d76..0b4689717b5e8 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -61,12 +61,6 @@ under the License. ${project.version} provided - - org.apache.flink - flink-java - ${project.version} - provided - org.apache.flink flink-streaming-java @@ -613,6 +607,10 @@ under the License. org.apache.flink flink-test-utils + + org.apache.flink + flink-test-utils-junit + ${project.build.directory}/test-dependencies diff --git a/flink-python/pyflink/common/execution_config.py b/flink-python/pyflink/common/execution_config.py index caad612659e7e..db46d95755130 100644 --- a/flink-python/pyflink/common/execution_config.py +++ b/flink-python/pyflink/common/execution_config.py @@ -509,15 +509,6 @@ def get_registered_pojo_types(self) -> List[str]: j_clz_set = self._j_execution_config.getRegisteredPojoTypes() return [value.getName() for value in j_clz_set] - def is_auto_type_registration_disabled(self) -> bool: - """ - Returns whether Flink is automatically registering all types in the user programs with - Kryo. - - :return: ``True`` means auto type registration is disabled and ``False`` means enabled. - """ - return self._j_execution_config.isAutoTypeRegistrationDisabled() - def is_use_snapshot_compression(self) -> bool: """ Returns whether he compression (snappy) for keyed state in full checkpoints and savepoints diff --git a/flink-python/pyflink/datastream/stream_execution_environment.py b/flink-python/pyflink/datastream/stream_execution_environment.py index 46c8e73fd86ba..3e507d1be5698 100644 --- a/flink-python/pyflink/datastream/stream_execution_environment.py +++ b/flink-python/pyflink/datastream/stream_execution_environment.py @@ -809,21 +809,6 @@ def from_source(self, j_type_info) return DataStream(j_data_stream=j_data_stream) - def read_text_file(self, file_path: str, charset_name: str = "UTF-8") -> DataStream: - """ - Reads the given file line-by-line and creates a DataStream that contains a string with the - contents of each such line. The charset with the given name will be used to read the files. - - Note that this interface is not fault tolerant that is supposed to be used for test purpose. - - :param file_path: The path of the file, as a URI (e.g., "file:///some/local/file" or - "hdfs://host:port/file/path") - :param charset_name: The name of the character set used to read the file. - :return: The DataStream that represents the data read from the given file as text lines. - """ - return DataStream(self._j_stream_execution_environment - .readTextFile(file_path, charset_name)) - def from_collection(self, collection: List[Any], type_info: TypeInformation = None) -> DataStream: """ diff --git a/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py b/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py index dda4b130fb88a..03d43e5fb3fe1 100644 --- a/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py +++ b/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py @@ -249,22 +249,6 @@ def test_add_custom_source(self): expected.sort() self.assertEqual(expected, results) - def test_read_text_file(self): - texts = ["Mike", "Marry", "Ted", "Jack", "Bob", "Henry"] - text_file_path = self.tempdir + '/text_file' - with open(text_file_path, 'a') as f: - for text in texts: - f.write(text) - f.write('\n') - - ds = self.env.read_text_file(text_file_path) - ds.add_sink(self.test_sink) - self.env.execute("test read text file") - results = self.test_sink.get_results() - results.sort() - texts.sort() - self.assertEqual(texts, results) - def test_execute_async(self): ds = self.env.from_collection([(1, 'Hi', 'Hello'), (2, 'Hello', 'Hi')], type_info=Types.ROW( diff --git a/flink-python/setup.py b/flink-python/setup.py index e5e1e231b22f2..3b03f41c7c779 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -197,8 +197,8 @@ def extracted_output_files(base_dir, file_path, output_directory): PYFLINK_UDF_RUNNER_SH = "pyflink-udf-runner.sh" PYFLINK_UDF_RUNNER_BAT = "pyflink-udf-runner.bat" -in_flink_source = os.path.isfile("../flink-java/src/main/java/org/apache/flink/api/java/" - "ExecutionEnvironment.java") +in_flink_source = os.path.isfile("../flink-runtime/src/main/java/org/apache/flink/streaming" + "/api/environment/StreamExecutionEnvironment.java") try: if in_flink_source: diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java index 005fb04af0efd..2ea8d806cd368 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java @@ -18,10 +18,10 @@ package org.apache.flink.client.python; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.client.program.ProgramAbortException; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.parser.CommandLineParser; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,7 +69,11 @@ public static void main(String[] args) throws Throwable { // streaming and batch environments are always set at the same time, for streaming jobs we // can // also get its configuration from batch environments. - Configuration config = ExecutionEnvironment.getExecutionEnvironment().getConfiguration(); + Configuration config = + Configuration.fromMap( + StreamExecutionEnvironment.getExecutionEnvironment() + .getConfiguration() + .toMap()); // start gateway server GatewayServer gatewayServer = PythonEnvUtils.startGatewayServer(); diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java index 5634a2880c2e5..1d62f8b1e9bb8 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java @@ -18,10 +18,10 @@ package org.apache.flink.client.python; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.python.util.PythonDependencyUtils; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.functions.python.PythonFunction; import org.apache.flink.util.FileUtils; @@ -123,8 +123,10 @@ static PythonFunction getPythonFunction( String objectName = fullyQualifiedName.substring(splitIndex + 1); Configuration mergedConfig = - new Configuration( - ExecutionEnvironment.getExecutionEnvironment().getConfiguration()); + Configuration.fromMap( + StreamExecutionEnvironment.getExecutionEnvironment() + .getConfiguration() + .toMap()); if (config instanceof TableConfig) { PythonDependencyUtils.merge(mergedConfig, ((TableConfig) config).getConfiguration()); } else { diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonTypeUtils.java index d01dab2296fee..8a44c16927d15 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/PythonTypeUtils.java @@ -54,7 +54,6 @@ import org.apache.flink.api.common.typeutils.base.array.IntPrimitiveArraySerializer; import org.apache.flink.api.common.typeutils.base.array.LongPrimitiveArraySerializer; import org.apache.flink.api.common.typeutils.base.array.ShortPrimitiveArraySerializer; -import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.api.java.typeutils.MapTypeInfo; @@ -66,6 +65,7 @@ import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.fnexecution.v1.FlinkFnApi; +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat; import org.apache.flink.streaming.api.typeinfo.python.PickledByteArrayTypeInfo; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; diff --git a/flink-python/src/main/java/org/apache/flink/table/utils/python/PythonTableUtils.java b/flink-python/src/main/java/org/apache/flink/table/utils/python/PythonTableUtils.java index 1d1ed0aab20f0..01dfab186dddc 100644 --- a/flink-python/src/main/java/org/apache/flink/table/utils/python/PythonTableUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/utils/python/PythonTableUtils.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.java.io.CollectionInputFormat; +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableDescriptor; diff --git a/flink-python/src/test/java/org/apache/flink/client/cli/TestJob.java b/flink-python/src/test/java/org/apache/flink/client/cli/TestJob.java index 9ca10d69665e2..2d5a116b6696d 100644 --- a/flink-python/src/test/java/org/apache/flink/client/cli/TestJob.java +++ b/flink-python/src/test/java/org/apache/flink/client/cli/TestJob.java @@ -18,11 +18,11 @@ package org.apache.flink.client.cli; -import org.apache.flink.api.java.utils.ParameterTool; 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; /** Test job which is used for {@link PythonProgramOptionsTest}. */ public class TestJob { diff --git a/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java b/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java index 9f054c1401881..d59ad6939260c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java +++ b/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java @@ -22,13 +22,13 @@ import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction; +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.ProviderContext; diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java index 92c7df3b8ba6b..00b96e0bbf3d2 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.webmonitor.history; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.HistoryServerOptions; @@ -46,6 +45,7 @@ import org.apache.flink.util.FatalExitExceptionHandler; import org.apache.flink.util.FileUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.ParameterTool; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; import org.apache.flink.util.concurrent.ExecutorThreadFactory; diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java index 7c42878053d9b..fb847aee7fd31 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java @@ -247,7 +247,7 @@ void testRestHandlerExceptionThrownWithEagerSinks() throws Exception { final String exceptionMsg = invocationException.get().getMessage(); assertThat(exceptionMsg) - .contains("Job was submitted in detached mode."); + .contains("Job client must be a CoordinationRequestGateway."); return true; }); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java index 844e0cba72c83..af9fc56606bf7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java @@ -94,7 +94,7 @@ void testJarSubmission(@TempDir File uploadDir, @TempDir File temporaryFolder) final JobPlanInfo planResponse = showPlan(planHandler, storedJarName, restfulGateway); // we're only interested in the core functionality so checking for a small detail is // sufficient - assertThat(planResponse.getJsonPlan()).contains("TestProgram.java:28"); + assertThat(planResponse.getJsonPlan()).contains("\"name\":\"Flink Streaming Job\""); runJar(runHandler, storedJarName, restfulGateway); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/EagerSinkProgram.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/EagerSinkProgram.java index 9f668113f74c3..1787de3de4505 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/EagerSinkProgram.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/EagerSinkProgram.java @@ -18,12 +18,12 @@ package org.apache.flink.runtime.webmonitor.handlers.utils; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** Javadoc. */ public class EagerSinkProgram { public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements("hello", "world").print(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.fromData("hello", "world").executeAndCollect(); } } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java index faf3af78add9b..9c5fca2de49b8 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/utils/TestProgram.java @@ -18,14 +18,14 @@ package org.apache.flink.runtime.webmonitor.handlers.utils; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; /** Simple test program. */ public class TestProgram { public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements("hello", "world").output(new DiscardingOutputFormat<>()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.fromData("hello", "world").sinkTo(new DiscardingSink<>()); env.execute(); } } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java index 9faeb2fef2729..b5047fc1b05d4 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.webmonitor.testutils; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; /** Simple test program that exposes passed arguments. */ public class ParameterProgram { @@ -29,8 +29,8 @@ public class ParameterProgram { public static void main(String[] args) throws Exception { actualArguments = args; - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements("hello", "world").output(new DiscardingOutputFormat<>()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.fromData("hello", "world").sinkTo(new DiscardingSink<>()); env.execute(); } } diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 91a944d0eb476..7e965ae402281 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -85,12 +85,6 @@ under the License. test - - org.apache.flink - flink-java - ${project.version} - - org.apache.flink flink-queryable-state-client-java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java index 3dce6c3d2f2a9..71573bb54cb08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java @@ -28,7 +28,6 @@ import org.apache.flink.api.common.typeutils.TypeComparatorFactory; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; -import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DelegatingConfiguration; import org.apache.flink.core.memory.DataInputViewStreamWrapper; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java index c59a5c92f0636..7781b62605f15 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java @@ -18,9 +18,9 @@ package org.apache.flink.runtime.zookeeper; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.util.ParameterTool; import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.ServerConfig; import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.ZooKeeperServer; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java index b09d842f6ac40..af2d1eb931fe8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java @@ -28,7 +28,6 @@ import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.NullByteKeySelector; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -60,6 +59,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.time.Duration; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java index 757932e2b3970..8f4573b125494 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/BroadcastConnectedStream.java @@ -22,7 +22,6 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction; @@ -30,6 +29,7 @@ import org.apache.flink.streaming.api.transformations.BroadcastStateTransformation; import org.apache.flink.streaming.api.transformations.KeyedBroadcastStateTransformation; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.List; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java index ddd02b182e686..16b9fae928433 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java @@ -23,13 +23,13 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java index d78d10c68f054..2842196a9b292 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -39,6 +38,7 @@ import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator; import org.apache.flink.streaming.api.operators.co.LegacyKeyedCoProcessOperator; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; +import org.apache.flink.util.Utils; import static java.util.Objects.requireNonNull; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index b8631fbd0c2c9..9ae200ba0c422 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -44,17 +44,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.dag.Transformation; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.io.CsvOutputFormat; -import org.apache.flink.api.java.io.TextOutputFormat; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.RpcOptions; import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; @@ -100,6 +95,7 @@ import org.apache.flink.streaming.util.keys.KeySelectorUtil; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.ArrayList; import java.util.List; @@ -873,120 +869,6 @@ public DataStreamSink printToErr(String sinkIdentifier) { return addSink(printFunction).name("Print to Std. Err"); } - /** - * Writes a DataStream to the file specified by path in text format. - * - *

For every element of the DataStream the result of {@link Object#toString()} is written. - * - * @param path The path pointing to the location the text file is written to. - * @return The closed DataStream. - * @deprecated Please use the {@link - * org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink} - * explicitly using the {@link #addSink(SinkFunction)} method. - */ - @PublicEvolving - public DataStreamSink writeAsText(String path) { - return writeUsingOutputFormat(new TextOutputFormat(new Path(path))); - } - - /** - * Writes a DataStream to the file specified by path in text format. - * - *

For every element of the DataStream the result of {@link Object#toString()} is written. - * - * @param path The path pointing to the location the text file is written to - * @param writeMode Controls the behavior for existing files. Options are NO_OVERWRITE and - * OVERWRITE. - * @return The closed DataStream. - * @deprecated Please use the {@link - * org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink} - * explicitly using the {@link #addSink(SinkFunction)} method. - */ - @Internal - public DataStreamSink writeAsText(String path, WriteMode writeMode) { - TextOutputFormat tof = new TextOutputFormat<>(new Path(path)); - tof.setWriteMode(writeMode); - return writeUsingOutputFormat(tof); - } - - /** - * Writes a DataStream to the file specified by the path parameter. - * - *

For every field of an element of the DataStream the result of {@link Object#toString()} is - * written. This method can only be used on data streams of tuples. - * - * @param path the path pointing to the location the text file is written to - * @return the closed DataStream - * @deprecated Please use the {@link - * org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink} - * explicitly using the {@link #addSink(SinkFunction)} method. - */ - @PublicEvolving - public DataStreamSink writeAsCsv(String path) { - return writeAsCsv( - path, - null, - CsvOutputFormat.DEFAULT_LINE_DELIMITER, - CsvOutputFormat.DEFAULT_FIELD_DELIMITER); - } - - /** - * Writes a DataStream to the file specified by the path parameter. - * - *

For every field of an element of the DataStream the result of {@link Object#toString()} is - * written. This method can only be used on data streams of tuples. - * - * @param path the path pointing to the location the text file is written to - * @param writeMode Controls the behavior for existing files. Options are NO_OVERWRITE and - * OVERWRITE. - * @return the closed DataStream - * @deprecated Please use the {@link - * org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink} - * explicitly using the {@link #addSink(SinkFunction)} method. - */ - @Internal - public DataStreamSink writeAsCsv(String path, WriteMode writeMode) { - return writeAsCsv( - path, - writeMode, - CsvOutputFormat.DEFAULT_LINE_DELIMITER, - CsvOutputFormat.DEFAULT_FIELD_DELIMITER); - } - - /** - * Writes a DataStream to the file specified by the path parameter. The writing is performed - * periodically every millis milliseconds. - * - *

For every field of an element of the DataStream the result of {@link Object#toString()} is - * written. This method can only be used on data streams of tuples. - * - * @param path the path pointing to the location the text file is written to - * @param writeMode Controls the behavior for existing files. Options are NO_OVERWRITE and - * OVERWRITE. - * @param rowDelimiter the delimiter for two rows - * @param fieldDelimiter the delimiter for two fields - * @return the closed DataStream - * @deprecated Please use the {@link - * org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink} - * explicitly using the {@link #addSink(SinkFunction)} method. - */ - @SuppressWarnings("unchecked") - @Internal - public DataStreamSink writeAsCsv( - String path, WriteMode writeMode, String rowDelimiter, String fieldDelimiter) { - Preconditions.checkArgument( - getType().isTupleType(), - "The writeAsCsv() method can only be used on data streams of tuples."); - - CsvOutputFormat of = new CsvOutputFormat<>(new Path(path), rowDelimiter, fieldDelimiter); - - if (writeMode != null) { - of.setWriteMode(writeMode); - } - - return writeUsingOutputFormat((OutputFormat) of); - } - /** * Writes the DataStream to a socket as a byte array. The format of the output is specified by a * {@link SerializationSchema}. diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java index e99b149d2d0f7..11777fd415733 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java @@ -23,9 +23,9 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java index 902a3b66e47d5..46ecad182b3bf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java @@ -28,7 +28,6 @@ import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.EnumTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; @@ -63,6 +62,7 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import org.apache.commons.lang3.StringUtils; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 986d13c65594a..5f9101162b792 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.java.RemoteEnvironmentConfigUtils; import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; @@ -30,6 +29,7 @@ import org.apache.flink.core.execution.PipelineExecutorServiceLoader; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.util.RemoteEnvironmentConfigUtils; import java.net.URL; import java.util.Arrays; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index a3881a246b111..1fae6a4e3bf6d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -34,7 +34,6 @@ import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.operators.SlotSharingGroup; import org.apache.flink.api.common.operators.util.SlotSharingGroupUtils; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; @@ -42,8 +41,6 @@ import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.ClosureCleaner; -import org.apache.flink.api.java.Utils; -import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.MissingTypeInfo; @@ -108,6 +105,7 @@ import org.apache.flink.util.SplittableIterator; import org.apache.flink.util.StringUtils; import org.apache.flink.util.TernaryBoolean; +import org.apache.flink.util.Utils; import org.apache.flink.util.WrappingRuntimeException; import javax.annotation.Nullable; @@ -1265,73 +1263,6 @@ private DataStreamSource fromParallelCollection( Boundedness.BOUNDED); } - /** - * Reads the given file line-by-line and creates a data stream that contains a string with the - * contents of each such line. The file will be read with the UTF-8 character set. - * - *

NOTES ON CHECKPOINTING: The source monitors the path, creates the {@link - * org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, forwards them to - * the downstream readers to read the actual data, and exits, without waiting for the readers to - * finish reading. This implies that no more checkpoint barriers are going to be forwarded after - * the source exits, thus having no checkpoints after that point. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or - * "hdfs://host:port/file/path"). - * @return The data stream that represents the data read from the given file as text lines - * @deprecated Use {@code - * FileSource#forRecordStreamFormat()/forBulkFileFormat()/forRecordFileFormat() instead}. An - * example of reading a file using a simple {@code TextLineInputFormat}: - *

{@code
-     * FileSource source =
-     *        FileSource.forRecordStreamFormat(
-     *           new TextLineInputFormat(), new Path("/foo/bar"))
-     *        .build();
-     * }
- */ - @Deprecated - public DataStreamSource readTextFile(String filePath) { - return readTextFile(filePath, "UTF-8"); - } - - /** - * Reads the given file line-by-line and creates a data stream that contains a string with the - * contents of each such line. The {@link java.nio.charset.Charset} with the given name will be - * used to read the files. - * - *

NOTES ON CHECKPOINTING: The source monitors the path, creates the {@link - * org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, forwards them to - * the downstream readers to read the actual data, and exits, without waiting for the readers to - * finish reading. This implies that no more checkpoint barriers are going to be forwarded after - * the source exits, thus having no checkpoints after that point. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or - * "hdfs://host:port/file/path") - * @param charsetName The name of the character set used to read the file - * @return The data stream that represents the data read from the given file as text lines - * @deprecated Use {@code - * FileSource#forRecordStreamFormat()/forBulkFileFormat()/forRecordFileFormat() instead}. An - * example of reading a file using a simple {@code TextLineInputFormat}: - *

{@code
-     * FileSource source =
-     *        FileSource.forRecordStreamFormat(
-     *         new TextLineInputFormat("UTF-8"), new Path("/foo/bar"))
-     *        .build();
-     * }
- */ - @Deprecated - public DataStreamSource readTextFile(String filePath, String charsetName) { - Preconditions.checkArgument( - !StringUtils.isNullOrWhitespaceOnly(filePath), - "The file path must not be null or blank."); - - TextInputFormat format = new TextInputFormat(new Path(filePath)); - format.setFilesFilter(FilePathFilter.createDefaultFilter()); - TypeInformation typeInfo = BasicTypeInfo.STRING_TYPE_INFO; - format.setCharsetName(charsetName); - - return readFile(format, filePath, FileProcessingMode.PROCESS_ONCE, -1, typeInfo); - } - /** * Reads the contents of the user-specified {@code filePath} based on the given {@link * FileInputFormat}. diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java index 91e89cc2170a3..2397e28cb9c97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.AggregateFunction; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java index 56d1732edbe9f..9d95bbd23e3e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.AggregateFunction; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyAllWindowFunction.java index 6572237f01e84..49d270d364f9e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyAllWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyWindowFunction.java index 4858f59b55f65..2e4ce6c564600 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java index 4980d61ec4378..80fd72e4ebfc8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java index 5c028de382606..5b6335bc94aaa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java index b3743f9dc0b89..9e963f6db4579 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java index 8bce2dd8caf1c..ce6c415d9775f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java index eb52bd6e0ba89..9d3d7e525a35d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java index 8608b1f0c34fe..628ac6232e88d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java index 2cd96fc3ab2ae..1ce36db0f2314 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java index 5fb2c4d077135..bf56f24d2e434 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java index 0a0fdf5c1b99f..21d2e15db6607 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java index 236073b192c80..388f26853e383 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.operators.translation.WrappingFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironmentConfigUtils.java b/flink-runtime/src/main/java/org/apache/flink/streaming/util/RemoteEnvironmentConfigUtils.java similarity index 95% rename from flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironmentConfigUtils.java rename to flink-runtime/src/main/java/org/apache/flink/streaming/util/RemoteEnvironmentConfigUtils.java index 60bdfe542fd06..d0ecefa076669 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironmentConfigUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/util/RemoteEnvironmentConfigUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.java; +package org.apache.flink.streaming.util; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.InvalidProgramException; @@ -25,6 +25,7 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.JarUtils; import java.io.File; @@ -48,7 +49,7 @@ public class RemoteEnvironmentConfigUtils { public static void validate(final String host, final int port) { - if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) { + if (!StreamExecutionEnvironment.areExplicitEnvironmentsAllowed()) { throw new InvalidProgramException( "The RemoteEnvironment cannot be instantiated when running in a pre-defined context " + "(such as Command Line Client or TestEnvironment)"); diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/util/functions/StreamingFunctionUtils.java b/flink-runtime/src/main/java/org/apache/flink/streaming/util/functions/StreamingFunctionUtils.java index 1ca9815855000..f05355cdc2c9a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/streaming/util/functions/StreamingFunctionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/streaming/util/functions/StreamingFunctionUtils.java @@ -21,10 +21,10 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.runtime.state.DefaultOperatorStateBackend; import org.apache.flink.runtime.state.JavaSerializer; import org.apache.flink.runtime.state.OperatorStateBackend; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/InputOutputFormatContainerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/InputOutputFormatContainerTest.java index cc9c5aa5fde95..21bcd8c9c6070 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/InputOutputFormatContainerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/InputOutputFormatContainerTest.java @@ -22,13 +22,13 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.operators.util.UserCodeWrapper; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.runtime.operators.util.TaskConfig; import org.junit.Test; +import java.io.IOException; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -47,11 +47,11 @@ public void testInputOutputFormat() { formatContainer.addParameters(operatorID1, "parameter1", "abc123"); OperatorID operatorID2 = new OperatorID(); - formatContainer.addOutputFormat(operatorID2, new DiscardingOutputFormat()); + formatContainer.addOutputFormat(operatorID2, new TestOutputFormat<>()); formatContainer.addParameters(operatorID2, "parameter1", "bcd234"); OperatorID operatorID3 = new OperatorID(); - formatContainer.addOutputFormat(operatorID3, new DiscardingOutputFormat()); + formatContainer.addOutputFormat(operatorID3, new TestOutputFormat()); formatContainer.addParameters(operatorID3, "parameter1", "cde345"); TaskConfig taskConfig = new TaskConfig(new Configuration()); @@ -77,16 +77,12 @@ public void testInputOutputFormat() { assertEquals("abc123", inputFormatParams.getString("parameter1", null)); // verify the output formats - assertTrue( - outputFormats.get(operatorID2).getUserCodeObject() - instanceof DiscardingOutputFormat); + assertTrue(outputFormats.get(operatorID2).getUserCodeObject() instanceof TestOutputFormat); Configuration outputFormatParams1 = loadedFormatContainer.getParameters(operatorID2); assertEquals(1, outputFormatParams1.keySet().size()); assertEquals("bcd234", outputFormatParams1.getString("parameter1", null)); - assertTrue( - outputFormats.get(operatorID3).getUserCodeObject() - instanceof DiscardingOutputFormat); + assertTrue(outputFormats.get(operatorID3).getUserCodeObject() instanceof TestOutputFormat); Configuration outputFormatParams2 = loadedFormatContainer.getParameters(operatorID3); assertEquals(1, outputFormatParams2.keySet().size()); assertEquals("cde345", outputFormatParams2.getString("parameter1", null)); @@ -127,7 +123,7 @@ public void testOnlyOutputFormat() { new InputOutputFormatContainer(Thread.currentThread().getContextClassLoader()); OperatorID operatorID = new OperatorID(); - formatContainer.addOutputFormat(operatorID, new DiscardingOutputFormat<>()); + formatContainer.addOutputFormat(operatorID, new TestOutputFormat<>()); Configuration parameters = new Configuration(); parameters.setString("parameter1", "bcd234"); @@ -144,9 +140,7 @@ public void testOnlyOutputFormat() { assertEquals(1, outputFormats.size()); assertEquals(0, loadedFormatContainer.getInputFormats().size()); - assertTrue( - outputFormats.get(operatorID).getUserCodeObject() - instanceof DiscardingOutputFormat); + assertTrue(outputFormats.get(operatorID).getUserCodeObject() instanceof TestOutputFormat); Configuration loadedParameters = loadedFormatContainer.getParameters(operatorID); assertEquals(1, loadedParameters.keySet().size()); @@ -184,4 +178,21 @@ public GenericInputSplit[] createInputSplits(int numSplits) { return null; } } + + private static final class TestOutputFormat implements OutputFormat { + + private static final long serialVersionUID = 1L; + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(InitializationContext context) throws IOException {} + + @Override + public void writeRecord(T record) {} + + @Override + public void close() {} + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java index 82747bae060c7..c42645be93453 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.io.GenericInputFormat; import org.apache.flink.api.common.io.InitializeOnMaster; import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; @@ -297,8 +296,8 @@ public void configure(Configuration parameters) { } } - private static final class TestingOutputFormat extends DiscardingOutputFormat - implements InitializeOnMaster, FinalizeOnMaster { + private static final class TestingOutputFormat + implements InitializeOnMaster, FinalizeOnMaster, OutputFormat { private boolean isConfigured = false; @@ -348,6 +347,15 @@ public void configure(Configuration parameters) { } isConfigured = true; } + + @Override + public void open(InitializationContext context) throws IOException {} + + @Override + public void writeRecord(Object record) throws IOException {} + + @Override + public void close() throws IOException {} } private static class TestClassLoader extends URLClassLoader { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DispatcherProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DispatcherProcess.java index 14c9d03acd197..6199bc68c747b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DispatcherProcess.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/DispatcherProcess.java @@ -18,13 +18,13 @@ package org.apache.flink.runtime.testutils; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.util.ParameterTool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java index fb90f6011c027..ed638e07ae824 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithIndirection.java @@ -18,17 +18,18 @@ package org.apache.flink.runtime.util.jartestprogram; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** Filter with additional indirections. */ public class FilterWithIndirection { public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet input = env.fromElements("Please filter", "the words", "but not this"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStreamSource input = env.fromData("Please filter", "the words", "but not this"); - DataSet output = input.filter(UtilFunctionWrapper.UtilFunction.getWordFilter()); + DataStream output = input.filter(UtilFunctionWrapper.UtilFunction.getWordFilter()); output.print(); env.execute(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java index e375df5870a35..4ff556e17da5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithLambda.java @@ -19,18 +19,19 @@ package org.apache.flink.runtime.util.jartestprogram; import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -/** Filter with lambda that is directly passed to {@link DataSet#filter(FilterFunction)}. */ +/** Filter with lambda that is directly passed to {@link DataStream#filter(FilterFunction)}. */ public class FilterWithLambda { @SuppressWarnings("Convert2MethodRef") public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet input = env.fromElements("Please filter", "the words", "but not this"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStreamSource input = env.fromData("Please filter", "the words", "but not this"); - DataSet output = input.filter((v) -> WordFilter.filter(v)); + DataStream output = input.filter((v) -> WordFilter.filter(v)); output.print(); env.execute(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java index c893762dfa199..1a78a27b730f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/FilterWithMethodReference.java @@ -19,19 +19,20 @@ package org.apache.flink.runtime.util.jartestprogram; import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** A lambda filter using a static method. */ public class FilterWithMethodReference { public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet input = env.fromElements("Please filter", "the words", "but not this"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStreamSource input = env.fromData("Please filter", "the words", "but not this"); FilterFunction filter = WordFilter::filter; - DataSet output = input.filter(filter); + DataStream output = input.filter(filter); output.print(); env.execute(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/StaticData.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/StaticData.java index c0a36dc7b3df9..a5ec1fc790431 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/StaticData.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/StaticData.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.util.jartestprogram; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; public class StaticData { - public static DataSet getDefaultTextLineDataSet(ExecutionEnvironment env) { + public static DataStreamSource getDefaultTextLineDataSet( + StreamExecutionEnvironment env) { - return env.fromElements( + return env.fromData( "To be, or not to be,--that is the question:--", "Whether 'tis nobler in the mind to suffer", "The slings and arrows of outrageous fortune", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithAnonymousClass.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithAnonymousClass.java index 113d94afa0e85..a4293d6dd6dbf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithAnonymousClass.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithAnonymousClass.java @@ -19,22 +19,24 @@ package org.apache.flink.runtime.util.jartestprogram; import org.apache.flink.api.common.functions.FlatMapFunction; -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.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; public class WordCountWithAnonymousClass { public static void main(String[] args) throws Exception { // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // get input data - DataSet text = StaticData.getDefaultTextLineDataSet(env); + DataStreamSource text = StaticData.getDefaultTextLineDataSet(env); - DataSet> counts = - // split up the lines in pairs (2-tuples) containing: (word,1) + // split up the lines in pairs (2-tuples) containing: (word,1) + DataStream> counts = text.flatMap( new FlatMapFunction>() { @Override @@ -52,8 +54,8 @@ public void flatMap( } } }) - // 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((KeySelector, String>) value -> value.f0) .sum(1); // emit result diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass.java index bfc9b2a21467a..03c47ad3dd5fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass.java @@ -18,24 +18,26 @@ package org.apache.flink.runtime.util.jartestprogram; -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.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; public class WordCountWithExternalClass { public static void main(String[] args) throws Exception { // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // get input data - DataSet text = StaticData.getDefaultTextLineDataSet(env); + DataStreamSource text = StaticData.getDefaultTextLineDataSet(env); - DataSet> counts = + DataStream> counts = // split up the lines in pairs (2-tuples) containing: (word,1) text.flatMap(new ExternalTokenizer()) - // 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((KeySelector, String>) value -> value.f0) .sum(1); // emit result diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass2.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass2.java index 04dabcc4794eb..e5833f7778df3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass2.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithExternalClass2.java @@ -18,24 +18,26 @@ package org.apache.flink.runtime.util.jartestprogram; -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.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; public class WordCountWithExternalClass2 { public static void main(String[] args) throws Exception { // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // get input data - DataSet text = StaticData.getDefaultTextLineDataSet(env); + DataStreamSource text = StaticData.getDefaultTextLineDataSet(env); - DataSet> counts = + DataStream> counts = // split up the lines in pairs (2-tuples) containing: (word,1) text.flatMap(new ExternalTokenizer2()) - // 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((KeySelector, String>) value -> value.f0) .sum(1); // emit result diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass.java index 393ef39333f89..9a5a1082fcb26 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/jartestprogram/WordCountWithInnerClass.java @@ -19,25 +19,27 @@ package org.apache.flink.runtime.util.jartestprogram; import org.apache.flink.api.common.functions.FlatMapFunction; -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.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; public class WordCountWithInnerClass { public static void main(String[] args) throws Exception { // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // get input data - DataSet text = StaticData.getDefaultTextLineDataSet(env); + DataStreamSource text = StaticData.getDefaultTextLineDataSet(env); - 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((KeySelector, String>) value -> value.f0) .sum(1); // emit result diff --git a/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/WrappingFunctionSnapshotRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/WrappingFunctionSnapshotRestoreTest.java index 6192316577d67..c570be690b37c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/WrappingFunctionSnapshotRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/streaming/api/operators/WrappingFunctionSnapshotRestoreTest.java @@ -20,10 +20,10 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.WrappingFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml index eee49d40539ce..91ac82b94c27d 100644 --- a/flink-streaming-java/pom.xml +++ b/flink-streaming-java/pom.xml @@ -71,12 +71,6 @@ under the License. ${project.version} - - org.apache.flink - flink-java - ${project.version} - - org.apache.flink flink-shaded-guava @@ -117,6 +111,12 @@ under the License. ${project.version} test + + + com.twitter + chill-java + ${chill.version} + diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java index 53f3e81dae801..a167d8cb17e3b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AsyncDataStream.java @@ -19,13 +19,13 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.functions.async.AsyncRetryStrategy; import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator; import org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorFactory; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Utils; import java.util.concurrent.TimeUnit; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/utils/CollectionInputFormat.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/CollectionInputFormat.java similarity index 90% rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/utils/CollectionInputFormat.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/CollectionInputFormat.java index 9e6eb6d5a51f6..801b94c4e9423 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/utils/CollectionInputFormat.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/CollectionInputFormat.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.factories.utils; +package org.apache.flink.streaming.api.legacy.io; import org.apache.flink.api.common.io.GenericInputFormat; import org.apache.flink.api.common.io.NonParallelInput; @@ -33,16 +33,7 @@ import java.util.Iterator; import java.util.List; -/** - * An input format that returns objects from a collection. - * - * @deprecated 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 +/** An input format that returns objects from a collection. */ public class CollectionInputFormat extends GenericInputFormat implements NonParallelInput { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/TextInputFormat.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/TextInputFormat.java new file mode 100644 index 0000000000000..83c8fc48cd6b7 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/TextInputFormat.java @@ -0,0 +1,107 @@ +/* + * 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.streaming.api.legacy.io; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.io.DelimitedInputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.Arrays; + +/** + * Input Format that reads text files. Each line results in another element. + * + * @deprecated If you want to read/write data from/to text file, please use FLIP-27 FileSource and + * Sink-V2 FlinkSink. This class should be removed if we drop the supports for read/write via + * input/output format in DataStream api. + */ +@Deprecated +@PublicEvolving +public class TextInputFormat extends DelimitedInputFormat { + + private static final long serialVersionUID = 1L; + + /** Code of \r, used to remove \r from a line when the line ends with \r\n. */ + private static final byte CARRIAGE_RETURN = (byte) '\r'; + + /** Code of \n, used to identify if \n is used as delimiter. */ + private static final byte NEW_LINE = (byte) '\n'; + + /** The name of the charset to use for decoding. */ + private String charsetName = "UTF-8"; + + // -------------------------------------------------------------------------------------------- + + public TextInputFormat(Path filePath) { + super(filePath, null); + } + + // -------------------------------------------------------------------------------------------- + + public String getCharsetName() { + return charsetName; + } + + public void setCharsetName(String charsetName) { + if (charsetName == null) { + throw new IllegalArgumentException("Charset must not be null."); + } + + this.charsetName = charsetName; + } + + // -------------------------------------------------------------------------------------------- + + @Override + public void configure(Configuration parameters) { + super.configure(parameters); + + if (charsetName == null || !Charset.isSupported(charsetName)) { + throw new RuntimeException("Unsupported charset: " + charsetName); + } + } + + // -------------------------------------------------------------------------------------------- + + @Override + public String readRecord(String reusable, byte[] bytes, int offset, int numBytes) + throws IOException { + // Check if \n is used as delimiter and the end of this line is a \r, then remove \r from + // the line + if (this.getDelimiter() != null + && this.getDelimiter().length == 1 + && this.getDelimiter()[0] == NEW_LINE + && offset + numBytes >= 1 + && bytes[offset + numBytes - 1] == CARRIAGE_RETURN) { + numBytes -= 1; + } + + return new String(bytes, offset, numBytes, this.charsetName); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public String toString() { + return "TextInputFormat (" + Arrays.toString(getFilePaths()) + ") - " + this.charsetName; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/TextOutputFormat.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/TextOutputFormat.java new file mode 100644 index 0000000000000..027607baa1f7d --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/legacy/io/TextOutputFormat.java @@ -0,0 +1,106 @@ +/* + * 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.streaming.api.legacy.io; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.io.FileOutputFormat; +import org.apache.flink.core.fs.Path; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.Charset; +import java.nio.charset.IllegalCharsetNameException; +import java.nio.charset.UnsupportedCharsetException; + +/** + * A {@link FileOutputFormat} that writes objects to a text file. + * + *

Objects are converted to Strings using either {@link Object#toString()} or a {@link + * TextFormatter}. + * + * @deprecated If you want to read/write data from/to text file, please use FLIP-27 FileSource and + * Sink-V2 FlinkSink. This class should be removed if we drop all supports for {@link + * org.apache.flink.api.common.io.InputFormat} and {@link + * org.apache.flink.api.common.io.OutputFormat} + */ +@Deprecated +@PublicEvolving +public class TextOutputFormat extends FileOutputFormat { + + private static final long serialVersionUID = 1L; + + private static final int NEWLINE = '\n'; + + private String charsetName; + + // -------------------------------------------------------------------------------------------- + + /** + * Formatter that transforms values into their {@link String} representations. + * + * @param type of input elements + */ + @PublicEvolving + public interface TextFormatter extends Serializable { + String format(IN value); + } + + public TextOutputFormat(Path outputPath) { + this(outputPath, "UTF-8"); + } + + public TextOutputFormat(Path outputPath, String charset) { + super(outputPath); + this.charsetName = charset; + } + + public String getCharsetName() { + return charsetName; + } + + public void setCharsetName(String charsetName) + throws IllegalCharsetNameException, UnsupportedCharsetException { + if (charsetName == null) { + throw new NullPointerException(); + } + + if (!Charset.isSupported(charsetName)) { + throw new UnsupportedCharsetException( + "The charset " + charsetName + " is not supported."); + } + + this.charsetName = charsetName; + } + + // -------------------------------------------------------------------------------------------- + + @Override + public void writeRecord(T record) throws IOException { + byte[] bytes = record.toString().getBytes(charsetName); + this.stream.write(bytes); + this.stream.write(NEWLINE); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public String toString() { + return "TextOutputFormat (" + getOutputFilePath() + ") - " + this.charsetName; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialize/FlinkChillPackageRegistrar.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialize/FlinkChillPackageRegistrar.java index 050206587334a..858d380c8228d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialize/FlinkChillPackageRegistrar.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialize/FlinkChillPackageRegistrar.java @@ -65,7 +65,7 @@ public int getNextRegistrationId() { @Override public void registerSerializers(Kryo kryo) { //noinspection ArraysAsListWithZeroOrOneArgument - new FlinkChillPackageRegistrar.RegistrationHelper(FIRST_REGISTRATION_ID, kryo) + new RegistrationHelper(FIRST_REGISTRATION_ID, kryo) .register(Arrays.asList("").getClass(), new ArraysAsListSerializer()) .register(BitSet.class, new BitSetSerializer()) .register(PriorityQueue.class, new PriorityQueueSerializer()) @@ -89,8 +89,7 @@ public RegistrationHelper(int firstRegistrationId, Kryo kryo) { this.kryo = kryo; } - public FlinkChillPackageRegistrar.RegistrationHelper register( - Class type, Serializer serializer) { + public RegistrationHelper register(Class type, Serializer serializer) { kryo.register(type, serializer, nextRegistrationId++); return this; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java index cc4d32fadd2c1..834ff119f1bf1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java @@ -45,14 +45,13 @@ import org.apache.flink.api.connector.source.lib.NumberSequenceSource; import org.apache.flink.api.connector.source.mocks.MockSource; import org.apache.flink.api.dag.Transformation; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.io.TypeSerializerInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.execution.CheckpointingMode; +import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.core.io.SimpleVersionedSerializerAdapter; import org.apache.flink.core.memory.ManagedMemoryUseCase; @@ -91,6 +90,8 @@ import org.apache.flink.streaming.api.functions.source.legacy.InputFormatSourceFunction; import org.apache.flink.streaming.api.functions.source.legacy.ParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; +import org.apache.flink.streaming.api.legacy.io.TextOutputFormat; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -124,6 +125,7 @@ import org.assertj.core.api.Assertions; import org.assertj.core.data.Offset; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -591,20 +593,23 @@ public void invoke(Tuple2 value) throws Exception {} } @Test - void testInputOutputFormat() { + void testInputOutputFormat(@TempDir java.nio.file.Path outputPath) { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataStream source = + DataStream source = env.addSource( new InputFormatSourceFunction<>( - new TypeSerializerInputFormat<>( - TypeInformation.of(Long.class)), - TypeInformation.of(Long.class)), - TypeInformation.of(Long.class)) + new TextInputFormat(new Path("FakePath")), + TypeInformation.of(String.class))) + .returns(TypeInformation.of(String.class)) .name("source"); - source.writeUsingOutputFormat(new DiscardingOutputFormat<>()).name("sink1"); - source.writeUsingOutputFormat(new DiscardingOutputFormat<>()).name("sink2"); + java.nio.file.Path outputFile1 = outputPath.resolve("outputFile1"); + java.nio.file.Path outputFile2 = outputPath.resolve("outputFile2"); + source.writeUsingOutputFormat(new TextOutputFormat<>(new Path(outputFile1.toUri()))) + .name("sink1"); + source.writeUsingOutputFormat(new TextOutputFormat<>(new Path(outputFile2.toUri()))) + .name("sink2"); StreamGraph streamGraph = env.getStreamGraph(); JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); @@ -637,15 +642,15 @@ void testInputOutputFormat() { InputFormat sourceFormat = inputFormats.get(nameToOperatorIds.get("Source: source")).getUserCodeObject(); - assertThat(sourceFormat).isInstanceOf(TypeSerializerInputFormat.class); + assertThat(sourceFormat).isInstanceOf(TextInputFormat.class); OutputFormat sinkFormat1 = outputFormats.get(nameToOperatorIds.get("Sink: sink1")).getUserCodeObject(); - assertThat(sinkFormat1).isInstanceOf(DiscardingOutputFormat.class); + assertThat(sinkFormat1).isInstanceOf(TextOutputFormat.class); OutputFormat sinkFormat2 = outputFormats.get(nameToOperatorIds.get("Sink: sink2")).getUserCodeObject(); - assertThat(sinkFormat2).isInstanceOf(DiscardingOutputFormat.class); + assertThat(sinkFormat2).isInstanceOf(TextOutputFormat.class); } @Test diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/TextInputFormatTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/legacy/io/TextInputFormatTest.java similarity index 99% rename from flink-java/src/test/java/org/apache/flink/api/java/io/TextInputFormatTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/api/legacy/io/TextInputFormatTest.java index a7403971c235d..14240ac2c144a 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/TextInputFormatTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/legacy/io/TextInputFormatTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.java.io; +package org.apache.flink.streaming.api.legacy.io; import org.apache.flink.api.common.io.compression.InflaterInputStreamFactory; import org.apache.flink.configuration.Configuration; diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java index 6c8d93391fdd6..2ece3c370048e 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java @@ -22,10 +22,13 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; import org.apache.flink.legacy.table.sinks.AppendStreamTableSink; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.legacy.io.TextOutputFormat; import org.apache.flink.table.legacy.api.TableSchema; import org.apache.flink.table.legacy.sinks.TableSink; import org.apache.flink.table.types.DataType; @@ -120,12 +123,13 @@ public DataStreamSink consumeDataStream(DataStream dataStream) { dataStream.map(new CsvFormatter(fieldDelim == null ? "," : fieldDelim)); DataStreamSink sink; + TextOutputFormat textOutputFormat = new TextOutputFormat<>(new Path(path)); if (writeMode != null) { - sink = csvRows.writeAsText(path, writeMode); - } else { - sink = csvRows.writeAsText(path); + textOutputFormat.setWriteMode(writeMode); } + sink = csvRows.addSink(new OutputFormatSinkFunction<>(textOutputFormat)); + if (numFiles > 0) { csvRows.setParallelism(numFiles); sink.setParallelism(numFiles); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index 334f0a4ee885e..c6763c8521aca 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -39,6 +39,7 @@ import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction; import org.apache.flink.streaming.api.functions.source.legacy.FromElementsFunction; import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction; +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogTable; @@ -107,7 +108,6 @@ import org.apache.flink.table.planner.factories.TestValuesRuntimeFunctions.TestNoLookupUntilNthAccessAsyncLookupFunction; import org.apache.flink.table.planner.factories.TestValuesRuntimeFunctions.TestNoLookupUntilNthAccessLookupFunction; import org.apache.flink.table.planner.factories.TestValuesRuntimeFunctions.TestValuesLookupFunction; -import org.apache.flink.table.planner.factories.utils.CollectionInputFormat; import org.apache.flink.table.planner.functions.aggfunctions.Count1AggFunction; import org.apache.flink.table.planner.functions.aggfunctions.CountAggFunction; import org.apache.flink.table.planner.functions.aggfunctions.MaxAggFunction; diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index 69981e00d1d54..b49474a9591f4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -26,6 +26,7 @@ import org.apache.flink.legacy.table.sources.StreamTableSource import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink, DataStreamSource} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction} import org.apache.flink.table.legacy.api.TableSchema diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala index 01dbffdb85623..e9bedb9db9d39 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala @@ -20,11 +20,11 @@ package org.apache.flink.table.planner.runtime.utils import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat import org.apache.flink.table.api._ import org.apache.flink.table.api.internal.TableEnvironmentImpl import org.apache.flink.table.expressions.Expression import org.apache.flink.table.planner.delegation.PlannerBase -import org.apache.flink.table.planner.factories.utils.CollectionInputFormat import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.utils.TableTestUtil diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TestLegacyLimitableTableSource.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TestLegacyLimitableTableSource.scala index 3f0bb63eaed6c..f3e6119f79dea 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TestLegacyLimitableTableSource.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TestLegacyLimitableTableSource.scala @@ -24,6 +24,7 @@ import org.apache.flink.legacy.table.factories.StreamTableSourceFactory import org.apache.flink.legacy.table.sources.StreamTableSource import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE @@ -31,7 +32,6 @@ import org.apache.flink.table.descriptors.DescriptorProperties import org.apache.flink.table.legacy.api.TableSchema import org.apache.flink.table.legacy.descriptors.Schema.SCHEMA import org.apache.flink.table.legacy.sources.{LimitableTableSource, TableSource} -import org.apache.flink.table.planner.factories.utils.CollectionInputFormat import org.apache.flink.table.utils.EncodingUtils import org.apache.flink.types.Row diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala index ae3cad5f1331d..14d314463b165 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/testTableSourceSinks.scala @@ -28,6 +28,7 @@ import org.apache.flink.legacy.table.sinks.StreamTableSink import org.apache.flink.legacy.table.sources.{InputFormatTableSource, StreamTableSource} import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat import org.apache.flink.table.api.{DataTypes, TableEnvironment} import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.catalog._ @@ -44,7 +45,6 @@ import org.apache.flink.table.legacy.sinks.TableSink import org.apache.flink.table.legacy.sources._ import org.apache.flink.table.legacy.sources.tsextractors.ExistingField import org.apache.flink.table.planner._ -import org.apache.flink.table.planner.factories.utils.CollectionInputFormat import org.apache.flink.table.planner.plan.hint.OptionsHintTest.IS_BOUNDED import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.EventTimeSourceFunction diff --git a/flink-test-utils-parent/flink-clients-test-utils/src/main/java/org/apache/flink/client/testjar/TestUserClassLoaderJob.java b/flink-test-utils-parent/flink-clients-test-utils/src/main/java/org/apache/flink/client/testjar/TestUserClassLoaderJob.java index 402817360d14e..df00a9ddbe568 100644 --- a/flink-test-utils-parent/flink-clients-test-utils/src/main/java/org/apache/flink/client/testjar/TestUserClassLoaderJob.java +++ b/flink-test-utils-parent/flink-clients-test-utils/src/main/java/org/apache/flink/client/testjar/TestUserClassLoaderJob.java @@ -18,11 +18,11 @@ package org.apache.flink.client.testjar; -import org.apache.flink.api.java.utils.ParameterTool; 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; /** This class can used to test situation that the jar is not in the system classpath. */ public class TestUserClassLoaderJob { diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java index ec595609437b8..c3d14e1c03dbb 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java @@ -18,12 +18,14 @@ package org.apache.flink.streaming.util; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.configuration.StateChangelogOptions; +import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -34,6 +36,8 @@ import java.time.Duration; import java.util.Collection; import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.runtime.testutils.PseudoRandomValueSelector.randomize; @@ -46,6 +50,12 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment { Boolean.parseBoolean(System.getProperty("checkpointing.randomization", "false")); private static final String STATE_CHANGE_LOG_CONFIG = System.getProperty("checkpointing.changelog", STATE_CHANGE_LOG_CONFIG_UNSET).trim(); + private static AtomicReference lastJobExecutionResult = + new AtomicReference<>(null); + private final MiniCluster miniCluster; + private final int parallelism; + private final Collection jarFiles; + private final Collection classPaths; public TestStreamEnvironment( MiniCluster miniCluster, @@ -60,6 +70,10 @@ public TestStreamEnvironment( null); setParallelism(parallelism); + this.miniCluster = miniCluster; + this.parallelism = parallelism; + this.jarFiles = jarFiles; + this.classPaths = classPaths; } public TestStreamEnvironment(MiniCluster miniCluster, int parallelism) { @@ -102,6 +116,22 @@ public static void setAsContext( initializeContextEnvironment(factory); } + public void setAsContext() { + StreamExecutionEnvironmentFactory factory = + conf -> { + TestStreamEnvironment env = + new TestStreamEnvironment( + miniCluster, conf, parallelism, jarFiles, classPaths); + + randomizeConfiguration(miniCluster, conf); + + env.configure(conf, env.getUserClassloader()); + return env; + }; + + initializeContextEnvironment(factory); + } + /** * This is the place for randomization the configuration that relates to DataStream API such as * ExecutionConf, CheckpointConf, StreamExecutionEnvironment. List of the configurations can be @@ -188,4 +218,24 @@ public static void setAsContext(final MiniCluster miniCluster, final int paralle public static void unsetAsContext() { resetContextEnvironment(); } + + @Override + public JobExecutionResult execute(String jobName) throws Exception { + JobExecutionResult result = super.execute(jobName); + this.lastJobExecutionResult.set(result); + return result; + } + + @Override + public JobClient executeAsync(String jobName) throws Exception { + JobClient jobClient = super.executeAsync(jobName); + CompletableFuture jobExecutionResultFuture = + jobClient.getJobExecutionResult(); + jobExecutionResultFuture.thenAccept((e) -> this.lastJobExecutionResult.set(e)); + return jobClient; + } + + public JobExecutionResult getLastJobExecutionResult() { + return lastJobExecutionResult.get(); + } } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/junit5/MiniClusterExtension.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/junit5/MiniClusterExtension.java index db95d8e9a63aa..976c118647389 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/junit5/MiniClusterExtension.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/junit5/MiniClusterExtension.java @@ -19,7 +19,6 @@ package org.apache.flink.test.junit5; import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.MiniClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; @@ -30,7 +29,6 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.TestStreamEnvironment; -import org.apache.flink.test.util.TestEnvironment; import org.junit.jupiter.api.TestFactory; import org.junit.jupiter.api.extension.AfterAllCallback; @@ -46,8 +44,8 @@ import java.util.function.Supplier; /** - * Starts a Flink {@link MiniCluster} and registers the respective {@link ExecutionEnvironment} and - * {@link StreamExecutionEnvironment} in the correct thread local environment. + * Starts a Flink {@link MiniCluster} and registers the {@link StreamExecutionEnvironment} in the + * correct thread local environment. * *

Example usage: * @@ -145,7 +143,7 @@ public final class MiniClusterExtension private InternalMiniClusterExtension internalMiniClusterExtension; - private TestEnvironment executionEnvironment; + private TestStreamEnvironment streamExecutionEnvironment; public MiniClusterExtension() { this( @@ -250,8 +248,8 @@ public void afterAll(ExtensionContext context) throws Exception { } } - public TestEnvironment getTestEnvironment() { - return this.executionEnvironment; + public TestStreamEnvironment getTestStreamEnvironment() { + return this.streamExecutionEnvironment; } // Implementation @@ -265,23 +263,16 @@ private void registerEnv(InternalMiniClusterExtension internalMiniClusterExtensi .getOptional(CoreOptions.DEFAULT_PARALLELISM) .orElse(internalMiniClusterExtension.getNumberSlots()); - TestEnvironment executionEnvironment = - new TestEnvironment( - internalMiniClusterExtension.getMiniCluster(), defaultParallelism, false); - executionEnvironment.setAsContext(); - this.executionEnvironment = - new TestEnvironment( - internalMiniClusterExtension.getMiniCluster(), - internalMiniClusterExtension.getNumberSlots(), - false); - this.executionEnvironment.setAsContext(); TestStreamEnvironment.setAsContext( internalMiniClusterExtension.getMiniCluster(), defaultParallelism); + this.streamExecutionEnvironment = + new TestStreamEnvironment( + internalMiniClusterExtension.getMiniCluster(), + internalMiniClusterExtension.getNumberSlots()); } private void unregisterEnv(InternalMiniClusterExtension internalMiniClusterExtension) { TestStreamEnvironment.unsetAsContext(); - TestEnvironment.unsetAsContext(); } private MiniClusterClient createMiniClusterClient( diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/CollectionTestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/CollectionTestEnvironment.java deleted file mode 100644 index 92bfd73b22b7d..0000000000000 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/CollectionTestEnvironment.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.test.util; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.java.CollectionEnvironment; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.ExecutionEnvironmentFactory; - -/** - * A {@link CollectionEnvironment} to be used in tests. The predominant feature of this class is - * that it allows setting it as a context environment, causing it to be returned by {@link - * ExecutionEnvironment#getExecutionEnvironment()}. This also allows retrieving the {@link - * JobExecutionResult} outside the actual program. - */ -public class CollectionTestEnvironment extends CollectionEnvironment { - - private CollectionTestEnvironment lastEnv = null; - - @Override - public JobExecutionResult getLastJobExecutionResult() { - if (lastEnv == null) { - return this.lastJobExecutionResult; - } else { - return lastEnv.getLastJobExecutionResult(); - } - } - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - JobExecutionResult result = super.execute(jobName); - this.lastJobExecutionResult = result; - return result; - } - - protected void setAsContext() { - ExecutionEnvironmentFactory factory = - new ExecutionEnvironmentFactory() { - @Override - public ExecutionEnvironment createExecutionEnvironment() { - lastEnv = new CollectionTestEnvironment(); - return lastEnv; - } - }; - - initializeContextEnvironment(factory); - } - - protected static void unsetAsContext() { - resetContextEnvironment(); - } -} diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java index b503d0967297b..69ace0e96d2ca 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java @@ -19,7 +19,8 @@ package org.apache.flink.test.util; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.jupiter.api.Test; @@ -70,7 +71,7 @@ public boolean isCollectionExecution() { // Methods to create the test program and for pre- and post- test work // -------------------------------------------------------------------------------------------- - protected abstract void testProgram() throws Exception; + protected abstract JobExecutionResult testProgram() throws Exception; protected void preSubmit() throws Exception {} @@ -94,18 +95,17 @@ public void testJobWithObjectReuse() throws Exception { // We should fix that we are able to get access to the latest execution result from a // different // execution environment and how the object reuse mode is enabled - TestEnvironment env = MINI_CLUSTER_EXTENSION.getTestEnvironment(); + TestStreamEnvironment env = MINI_CLUSTER_EXTENSION.getTestStreamEnvironment(); env.getConfig().enableObjectReuse(); // Possibly run the test multiple times executeProgramMultipleTimes(env); } - private void executeProgramMultipleTimes(ExecutionEnvironment env) throws Exception { + private void executeProgramMultipleTimes(StreamExecutionEnvironment env) throws Exception { for (int i = 0; i < numberOfTestRepetitions; i++) { try { - testProgram(); - this.latestExecutionResult = env.getLastJobExecutionResult(); + this.latestExecutionResult = testProgram(); } catch (Exception e) { System.err.println(e.getMessage()); e.printStackTrace(); @@ -129,44 +129,10 @@ public void testJobWithoutObjectReuse() throws Exception { // We should fix that we are able to get access to the latest execution result from a // different // execution environment and how the object reuse mode is enabled - ExecutionEnvironment env = MINI_CLUSTER_EXTENSION.getTestEnvironment(); + StreamExecutionEnvironment env = MINI_CLUSTER_EXTENSION.getTestStreamEnvironment(); env.getConfig().disableObjectReuse(); // Possibly run the test multiple times executeProgramMultipleTimes(env); } - - @Test - public void testJobCollectionExecution() throws Exception { - - // check if collection execution should be skipped. - if (this.skipCollectionExecution()) { - return; - } - - isCollectionExecution = true; - - preSubmit(); - // prepare the test environment - CollectionTestEnvironment env = new CollectionTestEnvironment(); - env.setAsContext(); - - // call the test program - try { - testProgram(); - this.latestExecutionResult = env.getLastJobExecutionResult(); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Error while calling the test program: " + e.getMessage()); - } finally { - MINI_CLUSTER_EXTENSION.getTestEnvironment().setAsContext(); - } - - assertThat(this.latestExecutionResult) - .as("The test program never triggered an execution.") - .isNotNull(); - - postSubmit(); - } } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBaseJUnit4.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBaseJUnit4.java index 98c888c6ad792..b4c4f80edcc17 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBaseJUnit4.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBaseJUnit4.java @@ -19,7 +19,7 @@ package org.apache.flink.test.util; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.util.TestStreamEnvironment; import org.junit.Assert; import org.junit.Test; @@ -49,24 +49,18 @@ public abstract class JavaProgramTestBaseJUnit4 extends AbstractTestBaseJUnit4 { */ private int numberOfTestRepetitions = 1; - private boolean isCollectionExecution; - public void setNumberOfTestRepetitions(int numberOfTestRepetitions) { this.numberOfTestRepetitions = numberOfTestRepetitions; } public int getParallelism() { - return isCollectionExecution ? 1 : MINI_CLUSTER_RESOURCE.getNumberSlots(); + return MINI_CLUSTER_RESOURCE.getNumberSlots(); } public JobExecutionResult getLatestExecutionResult() { return this.latestExecutionResult; } - public boolean isCollectionExecution() { - return isCollectionExecution; - } - // -------------------------------------------------------------------------------------------- // Methods to create the test program and for pre- and post- test work // -------------------------------------------------------------------------------------------- @@ -87,8 +81,6 @@ protected boolean skipCollectionExecution() { @Test public void testJobWithObjectReuse() throws Exception { - isCollectionExecution = false; - // pre-submit try { preSubmit(); @@ -102,7 +94,7 @@ public void testJobWithObjectReuse() throws Exception { // We should fix that we are able to get access to the latest execution result from a // different // execution environment and how the object reuse mode is enabled - TestEnvironment env = MINI_CLUSTER_RESOURCE.getTestEnvironment(); + TestStreamEnvironment env = MINI_CLUSTER_RESOURCE.getTestStreamEnvironment(); env.getConfig().enableObjectReuse(); // Possibly run the test multiple times @@ -133,8 +125,6 @@ public void testJobWithObjectReuse() throws Exception { @Test public void testJobWithoutObjectReuse() throws Exception { - isCollectionExecution = false; - // pre-submit try { preSubmit(); @@ -148,7 +138,7 @@ public void testJobWithoutObjectReuse() throws Exception { // We should fix that we are able to get access to the latest execution result from a // different // execution environment and how the object reuse mode is enabled - ExecutionEnvironment env = MINI_CLUSTER_RESOURCE.getTestEnvironment(); + TestStreamEnvironment env = MINI_CLUSTER_RESOURCE.getTestStreamEnvironment(); env.getConfig().disableObjectReuse(); // Possibly run the test multiple times @@ -176,52 +166,4 @@ public void testJobWithoutObjectReuse() throws Exception { Assert.fail("Post-submit work caused an error: " + e.getMessage()); } } - - @Test - public void testJobCollectionExecution() throws Exception { - - // check if collection execution should be skipped. - if (this.skipCollectionExecution()) { - return; - } - - isCollectionExecution = true; - - // pre-submit - try { - preSubmit(); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - Assert.fail("Pre-submit work caused an error: " + e.getMessage()); - } - - // prepare the test environment - CollectionTestEnvironment env = new CollectionTestEnvironment(); - env.setAsContext(); - - // call the test program - try { - testProgram(); - this.latestExecutionResult = env.getLastJobExecutionResult(); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - Assert.fail("Error while calling the test program: " + e.getMessage()); - } finally { - MINI_CLUSTER_RESOURCE.getTestEnvironment().setAsContext(); - } - - Assert.assertNotNull( - "The test program never triggered an execution.", this.latestExecutionResult); - - // post-submit - try { - postSubmit(); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - Assert.fail("Post-submit work caused an error: " + e.getMessage()); - } - } } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterWithClientResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterWithClientResource.java index 07ab415e21f32..4793af548f004 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterWithClientResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterWithClientResource.java @@ -26,16 +26,13 @@ import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.util.ExceptionUtils; -/** - * Starts a Flink mini cluster as a resource and registers the respective ExecutionEnvironment and - * StreamExecutionEnvironment. - */ +/** Starts a Flink mini cluster as a resource and registers the StreamExecutionEnvironment. */ public class MiniClusterWithClientResource extends MiniClusterResource { private ClusterClient clusterClient; private RestClusterClient restClusterClient; - private TestEnvironment executionEnvironment; + private TestStreamEnvironment streamExecutionEnvironment; public MiniClusterWithClientResource( final MiniClusterResourceConfiguration miniClusterResourceConfiguration) { @@ -55,8 +52,8 @@ public RestClusterClient getRestClusterClient() throws Exception { return restClusterClient; } - public TestEnvironment getTestEnvironment() { - return executionEnvironment; + public TestStreamEnvironment getTestStreamEnvironment() { + return streamExecutionEnvironment; } @Override @@ -66,16 +63,14 @@ public void before() throws Exception { clusterClient = createMiniClusterClient(); restClusterClient = createRestClusterClient(); - executionEnvironment = new TestEnvironment(getMiniCluster(), getNumberSlots(), false); - executionEnvironment.setAsContext(); TestStreamEnvironment.setAsContext(getMiniCluster(), getNumberSlots()); + streamExecutionEnvironment = new TestStreamEnvironment(getMiniCluster(), getNumberSlots()); } @Override public void after() { log.info("Finalization triggered: Cluster shutdown is going to be initiated."); TestStreamEnvironment.unsetAsContext(); - TestEnvironment.unsetAsContext(); Exception exception = null; diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java index 19628a09b32f1..5c0a79331f1e6 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.util; +import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; @@ -38,14 +39,14 @@ *

{@code
  * {@literal @}Test
  * public void someTest() {
- *     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ *     StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  *     // test code
  *     env.execute();
  * }
  *
  * {@literal @}Test
  * public void anotherTest() {
- *     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ *     StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  *     // test code
  *     env.execute();
  * }
@@ -60,8 +61,7 @@ public class MultipleProgramsTestBase extends AbstractTestBase {
      */
     public enum TestExecutionMode {
         CLUSTER,
-        CLUSTER_OBJECT_REUSE,
-        COLLECTION,
+        CLUSTER_OBJECT_REUSE
     }
 
     // ------------------------------------------------------------------------
@@ -74,24 +74,21 @@ public enum TestExecutionMode {
 
     @BeforeEach
     public void setupEnvironment() {
-        TestEnvironment testEnvironment;
+        TestStreamEnvironment testStreamEnvironment;
         switch (mode) {
             case CLUSTER:
                 // This only works because of the quirks we built in the TestEnvironment.
                 // We should refactor this in the future!!!
-                testEnvironment = MINI_CLUSTER_EXTENSION.getTestEnvironment();
-                testEnvironment.getConfig().disableObjectReuse();
-                testEnvironment.setAsContext();
+                testStreamEnvironment = MINI_CLUSTER_EXTENSION.getTestStreamEnvironment();
+                testStreamEnvironment.getConfig().disableObjectReuse();
+                testStreamEnvironment.setAsContext();
                 break;
             case CLUSTER_OBJECT_REUSE:
                 // This only works because of the quirks we built in the TestEnvironment.
                 // We should refactor this in the future!!!
-                testEnvironment = MINI_CLUSTER_EXTENSION.getTestEnvironment();
-                testEnvironment.getConfig().enableObjectReuse();
-                testEnvironment.setAsContext();
-                break;
-            case COLLECTION:
-                new CollectionTestEnvironment().setAsContext();
+                testStreamEnvironment = MINI_CLUSTER_EXTENSION.getTestStreamEnvironment();
+                testStreamEnvironment.getConfig().enableObjectReuse();
+                testStreamEnvironment.setAsContext();
                 break;
         }
     }
@@ -101,10 +98,7 @@ public void teardownEnvironment() {
         switch (mode) {
             case CLUSTER:
             case CLUSTER_OBJECT_REUSE:
-                TestEnvironment.unsetAsContext();
-                break;
-            case COLLECTION:
-                CollectionTestEnvironment.unsetAsContext();
+                TestStreamEnvironment.unsetAsContext();
                 break;
         }
     }
@@ -115,6 +109,6 @@ public void teardownEnvironment() {
 
     @Parameters(name = "Execution mode = {0}")
     public static Collection executionModes() {
-        return Arrays.asList(TestExecutionMode.CLUSTER, TestExecutionMode.COLLECTION);
+        return Arrays.asList(TestExecutionMode.CLUSTER);
     }
 }
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBaseJUnit4.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBaseJUnit4.java
index 40eac84b8c61c..6b3bc01d10a6b 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBaseJUnit4.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBaseJUnit4.java
@@ -18,12 +18,14 @@
 
 package org.apache.flink.test.util;
 
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.runners.Parameterized;
 
-import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 
 /**
  * Base class for unit tests that run multiple tests and want to reuse the same Flink cluster. This
@@ -36,14 +38,14 @@
  * 
{@code
  * {@literal @}Test
  * public void someTest() {
- *     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ *     StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  *     // test code
  *     env.execute();
  * }
  *
  * {@literal @}Test
  * public void anotherTest() {
- *     ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ *     StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  *     // test code
  *     env.execute();
  * }
@@ -61,8 +63,7 @@ public class MultipleProgramsTestBaseJUnit4 extends AbstractTestBaseJUnit4 {
      */
     public enum TestExecutionMode {
         CLUSTER,
-        CLUSTER_OBJECT_REUSE,
-        COLLECTION,
+        CLUSTER_OBJECT_REUSE
     }
 
     // ------------------------------------------------------------------------
@@ -79,24 +80,21 @@ public MultipleProgramsTestBaseJUnit4(TestExecutionMode mode) {
 
     @Before
     public void setupEnvironment() {
-        TestEnvironment testEnvironment;
+        TestStreamEnvironment testStreamEnvironment;
         switch (mode) {
             case CLUSTER:
-                // This only works because of the quirks we built in the TestEnvironment.
+                // This only works because of the quirks we built in the TestStreamEnvironment.
                 // We should refactor this in the future!!!
-                testEnvironment = MINI_CLUSTER_RESOURCE.getTestEnvironment();
-                testEnvironment.getConfig().disableObjectReuse();
-                testEnvironment.setAsContext();
+                testStreamEnvironment = MINI_CLUSTER_RESOURCE.getTestStreamEnvironment();
+                testStreamEnvironment.getConfig().disableObjectReuse();
+                testStreamEnvironment.setAsContext();
                 break;
             case CLUSTER_OBJECT_REUSE:
-                // This only works because of the quirks we built in the TestEnvironment.
+                // This only works because of the quirks we built in the TestStreamEnvironment.
                 // We should refactor this in the future!!!
-                testEnvironment = MINI_CLUSTER_RESOURCE.getTestEnvironment();
-                testEnvironment.getConfig().enableObjectReuse();
-                testEnvironment.setAsContext();
-                break;
-            case COLLECTION:
-                new CollectionTestEnvironment().setAsContext();
+                testStreamEnvironment = MINI_CLUSTER_RESOURCE.getTestStreamEnvironment();
+                testStreamEnvironment.getConfig().enableObjectReuse();
+                testStreamEnvironment.setAsContext();
                 break;
         }
     }
@@ -106,10 +104,7 @@ public void teardownEnvironment() {
         switch (mode) {
             case CLUSTER:
             case CLUSTER_OBJECT_REUSE:
-                TestEnvironment.unsetAsContext();
-                break;
-            case COLLECTION:
-                CollectionTestEnvironment.unsetAsContext();
+                TestStreamEnvironment.unsetAsContext();
                 break;
         }
     }
@@ -120,8 +115,6 @@ public void teardownEnvironment() {
 
     @Parameterized.Parameters(name = "Execution mode = {0}")
     public static Collection executionModes() {
-        return Arrays.asList(
-                new Object[] {TestExecutionMode.CLUSTER},
-                new Object[] {TestExecutionMode.COLLECTION});
+        return Collections.singletonList(new Object[] {TestExecutionMode.CLUSTER});
     }
 }
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
deleted file mode 100644
index b63d9c4d3f02a..0000000000000
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ /dev/null
@@ -1,138 +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.util;
-
-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.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.minicluster.MiniCluster;
-import org.apache.flink.util.Preconditions;
-
-import java.net.URL;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * A {@link ExecutionEnvironment} implementation which executes its jobs on a {@link MiniCluster}.
- */
-public class TestEnvironment extends ExecutionEnvironment {
-
-    private final MiniCluster miniCluster;
-
-    private TestEnvironment lastEnv;
-
-    public TestEnvironment(
-            MiniCluster miniCluster,
-            int parallelism,
-            boolean isObjectReuseEnabled,
-            Collection jarFiles,
-            Collection classPaths) {
-        super(
-                new MiniClusterPipelineExecutorServiceLoader(miniCluster),
-                MiniClusterPipelineExecutorServiceLoader.updateConfigurationForMiniCluster(
-                        new Configuration(), jarFiles, classPaths),
-                null);
-
-        this.miniCluster = Preconditions.checkNotNull(miniCluster);
-
-        setParallelism(parallelism);
-
-        if (isObjectReuseEnabled) {
-            getConfig().enableObjectReuse();
-        } else {
-            getConfig().disableObjectReuse();
-        }
-
-        lastEnv = null;
-    }
-
-    public TestEnvironment(MiniCluster executor, int parallelism, boolean isObjectReuseEnabled) {
-        this(
-                executor,
-                parallelism,
-                isObjectReuseEnabled,
-                Collections.emptyList(),
-                Collections.emptyList());
-    }
-
-    @Override
-    public JobExecutionResult getLastJobExecutionResult() {
-        if (lastEnv == null) {
-            return lastJobExecutionResult;
-        } else {
-            return lastEnv.getLastJobExecutionResult();
-        }
-    }
-
-    public void setAsContext() {
-        ExecutionEnvironmentFactory factory =
-                () -> {
-                    lastEnv =
-                            new TestEnvironment(
-                                    miniCluster,
-                                    getParallelism(),
-                                    getConfig().isObjectReuseEnabled());
-                    return lastEnv;
-                };
-
-        initializeContextEnvironment(factory);
-    }
-
-    // ---------------------------------------------------------------------------------------------
-
-    /**
-     * Sets the current {@link ExecutionEnvironment} to be a {@link TestEnvironment}. The test
-     * environment executes the given jobs on a Flink mini cluster with the given default
-     * parallelism and the additional jar files and class paths.
-     *
-     * @param miniCluster The MiniCluster to execute jobs on.
-     * @param parallelism The default parallelism
-     * @param jarFiles Additional jar files to execute the job with
-     * @param classPaths Additional class paths to execute the job with
-     */
-    public static void setAsContext(
-            final MiniCluster miniCluster,
-            final int parallelism,
-            final Collection jarFiles,
-            final Collection classPaths) {
-
-        ExecutionEnvironmentFactory factory =
-                () -> new TestEnvironment(miniCluster, parallelism, false, jarFiles, classPaths);
-
-        initializeContextEnvironment(factory);
-    }
-
-    /**
-     * Sets the current {@link ExecutionEnvironment} to be a {@link TestEnvironment}. The test
-     * environment executes the given jobs on a Flink mini cluster with the given default
-     * parallelism and the additional jar files and class paths.
-     *
-     * @param miniCluster The MiniCluster to execute jobs on.
-     * @param parallelism The default parallelism
-     */
-    public static void setAsContext(final MiniCluster miniCluster, final int parallelism) {
-        setAsContext(miniCluster, parallelism, Collections.emptyList(), Collections.emptyList());
-    }
-
-    public static void unsetAsContext() {
-        resetContextEnvironment();
-    }
-}
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 8fe3ec105fd91..9b829ea030234 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -125,13 +125,6 @@ under the License.
 			test
 			test-jar
 		
-
-		
-			org.apache.flink
-			flink-optimizer
-			${project.version}
-			test
-		
 		
 		
 			org.apache.flink
@@ -146,13 +139,6 @@ under the License.
 			${project.version}
 			test
 		
-		
-		
-			org.apache.flink
-			flink-java
-			${project.version}
-			test
-		
 
 		
 			org.apache.flink
@@ -165,20 +151,24 @@ under the License.
 			${project.version}
 			test
 		
-		
-		
-			org.apache.flink
-			flink-examples-batch
-			${project.version}
-			test
-		
 
 		
 			org.apache.flink
-			flink-java
+			flink-examples-streaming
 			${project.version}
-			test-jar
 			test
+			
+				
+				
+					org.apache.flink
+					flink-connector-kafka
+				
+			
 		
 
 		
@@ -196,14 +186,6 @@ under the License.
 			test
 		
 
-		
-			org.apache.flink
-			flink-optimizer
-			${project.version}
-			test-jar
-			test
-		
-
 		
 			org.apache.flink
 			flink-runtime
@@ -503,25 +485,6 @@ under the License.
 						
 					
 
-					
-						create-kmeans-jar
-						process-test-classes
-						
-							single
-						
-						
-							
-								
-									org.apache.flink.test.classloading.jar.KMeansForTest
-								
-							
-							kmeans
-							false
-							
-								src/test/assembly/test-kmeans-assembly.xml
-							
-						
-					
 					
 						create-usercodetype-jar
 						process-test-classes
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
index 9269981c095f3..5f60c2b07939f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
@@ -24,13 +24,13 @@
 import org.apache.flink.api.common.accumulators.LongCounter;
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
 import org.apache.flink.test.util.MiniClusterWithClientResource;
 import org.apache.flink.util.TestLogger;
 
@@ -66,25 +66,25 @@ public static Configuration getConfiguration() {
 
     @Test
     public void testFaultyAccumulator() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
         // Test Exception forwarding with faulty Accumulator implementation
-        env.generateSequence(0, 10000)
+        env.fromSequence(0, 10000)
                 .map(new FaultyAccumulatorUsingMapper())
-                .output(new DiscardingOutputFormat<>());
+                .sinkTo(new DiscardingSink<>());
 
         assertAccumulatorsShouldFail(env.execute());
     }
 
     @Test
     public void testInvalidTypeAccumulator() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
         // Test Exception forwarding with faulty Accumulator implementation
-        env.generateSequence(0, 10000)
+        env.fromSequence(0, 10000)
                 .map(new IncompatibleAccumulatorTypesMapper())
                 .map(new IncompatibleAccumulatorTypesMapper2())
-                .output(new DiscardingOutputFormat<>());
+                .sinkTo(new DiscardingSink<>());
 
         try {
             env.execute();
@@ -96,12 +96,12 @@ public void testInvalidTypeAccumulator() throws Exception {
 
     @Test
     public void testFaultyMergeAccumulator() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
         // Test Exception forwarding with faulty Accumulator implementation
-        env.generateSequence(0, 10000)
+        env.fromSequence(0, 10000)
                 .map(new FaultyMergeAccumulatorUsingMapper())
-                .output(new DiscardingOutputFormat<>());
+                .sinkTo(new DiscardingSink<>());
 
         assertAccumulatorsShouldFail(env.execute());
     }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
index d0d60264ee614..9b5c9f2902fc6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
@@ -25,12 +25,19 @@
 import org.apache.flink.api.common.accumulators.Histogram;
 import org.apache.flink.api.common.accumulators.IntCounter;
 import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-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.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.legacy.io.TextInputFormat;
+import org.apache.flink.streaming.api.legacy.io.TextOutputFormat;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.test.util.JavaProgramTestBaseJUnit4;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.Collector;
@@ -99,14 +106,35 @@ protected void postSubmit() throws Exception {
 
     @Override
     protected void testProgram() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-        DataSet input = env.readTextFile(dataPath);
+        DataStreamSource input =
+                env.createInput(new TextInputFormat(new Path(dataPath)))
+                        .setParallelism(getParallelism());
 
         input.flatMap(new TokenizeLine())
-                .groupBy(0)
-                .reduceGroup(new CountWords())
-                .writeAsCsv(resultPath, "\n", " ");
+                .keyBy(x -> x.f0)
+                .window(GlobalWindows.createWithEndOfStreamTrigger())
+                .reduce(
+                        new ReduceFunction>() {
+                            @Override
+                            public Tuple2 reduce(
+                                    Tuple2 value1, Tuple2 value2)
+                                    throws Exception {
+                                return Tuple2.of(value2.f0, value1.f1 + value2.f1);
+                            }
+                        })
+                .map(
+                        new MapFunction, String>() {
+
+                            @Override
+                            public String map(Tuple2 value) throws Exception {
+                                return value.f0 + " " + value.f1;
+                            }
+                        })
+                .addSink(
+                        new OutputFormatSinkFunction<>(
+                                new TextOutputFormat<>(new Path(resultPath))));
 
         this.result = env.execute();
     }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java
deleted file mode 100644
index 1eb80fe936c39..0000000000000
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java
+++ /dev/null
@@ -1,80 +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.accumulators;
-
-import org.apache.flink.api.common.accumulators.IntCounter;
-import org.apache.flink.api.common.functions.OpenContext;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.test.util.JavaProgramTestBaseJUnit4;
-import org.apache.flink.util.Collector;
-
-import org.junit.Assert;
-
-/** Test accumulator within iteration. */
-public class AccumulatorIterativeITCase extends JavaProgramTestBaseJUnit4 {
-    private static final int NUM_ITERATIONS = 3;
-    private static final int NUM_SUBTASKS = 1;
-    private static final String ACC_NAME = "test";
-
-    @Override
-    protected boolean skipCollectionExecution() {
-        return true;
-    }
-
-    @Override
-    protected void testProgram() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        env.setParallelism(NUM_SUBTASKS);
-
-        IterativeDataSet iteration = env.fromElements(1, 2, 3).iterate(NUM_ITERATIONS);
-
-        iteration
-                .closeWith(iteration.reduceGroup(new SumReducer()))
-                .output(new DiscardingOutputFormat());
-
-        Assert.assertEquals(NUM_ITERATIONS * 6, (int) env.execute().getAccumulatorResult(ACC_NAME));
-    }
-
-    static final class SumReducer extends RichGroupReduceFunction {
-
-        private static final long serialVersionUID = 1L;
-
-        private IntCounter testCounter = new IntCounter();
-
-        @Override
-        public void open(OpenContext openContext) throws Exception {
-            getRuntimeContext().addAccumulator(ACC_NAME, this.testCounter);
-        }
-
-        @Override
-        public void reduce(Iterable values, Collector out) {
-            // Compute the sum
-            int sum = 0;
-
-            for (Integer value : values) {
-                sum += value;
-                testCounter.add(value);
-            }
-            out.collect(sum);
-        }
-    }
-}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
index 803e55b68a646..c73419f719292 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
@@ -18,24 +18,18 @@
 
 package org.apache.flink.test.accumulators;
 
-import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.RuntimeExecutionMode;
 import org.apache.flink.api.common.accumulators.IntCounter;
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.io.OutputFormat;
 import org.apache.flink.api.common.time.Deadline;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HeartbeatManagerOptions;
 import org.apache.flink.configuration.RpcOptions;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.minicluster.MiniClusterJobClient;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
@@ -52,6 +46,8 @@
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,10 +62,13 @@
 import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult;
 
 /** Tests the availability of accumulator results during runtime. */
+@RunWith(Parameterized.class)
 public class AccumulatorLiveITCase extends TestLogger {
 
     private static final Logger LOG = LoggerFactory.getLogger(AccumulatorLiveITCase.class);
 
+    @Parameterized.Parameter public boolean testBatchJob;
+
     @ClassRule
     public static final TestExecutorResource EXECUTOR_RESOURCE =
             TestingUtils.defaultExecutorResource();
@@ -91,6 +90,11 @@ public class AccumulatorLiveITCase extends TestLogger {
         }
     }
 
+    @Parameterized.Parameters(name = "testBatchJob: {0}")
+    public static Object[] parameters() {
+        return new Object[] {true, false};
+    }
+
     @ClassRule
     public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE =
             new MiniClusterWithClientResource(
@@ -114,23 +118,11 @@ public void resetLatches() throws InterruptedException {
     }
 
     @Test
-    public void testBatch() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        env.setParallelism(1);
-
-        DataSet input = env.fromCollection(inputData);
-        input.flatMap(new NotifyingMapper()).output(new DummyOutputFormat());
-
-        // Extract job graph and set job id for the task to notify of accumulator changes.
-        JobGraph jobGraph = getJobGraph(env.createProgramPlan());
-
-        submitJobAndVerifyResults(jobGraph);
-    }
-
-    @Test
-    public void testStreaming() throws Exception {
-
+    public void testJob() throws Exception {
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        RuntimeExecutionMode runtimeExecutionMode =
+                testBatchJob ? RuntimeExecutionMode.BATCH : RuntimeExecutionMode.STREAMING;
+        env.setRuntimeMode(runtimeExecutionMode);
         env.setParallelism(1);
 
         DataStream input = env.fromData(inputData);
@@ -261,12 +253,4 @@ public void writeRecord(Integer record) throws IOException {}
         @Override
         public void close() throws IOException {}
     }
-
-    /** Helpers to generate the JobGraph. */
-    private static JobGraph getJobGraph(Plan plan) {
-        Optimizer pc = new Optimizer(new DataStatistics(), new Configuration());
-        JobGraphGenerator jgg = new JobGraphGenerator();
-        OptimizedPlan op = pc.compile(plan);
-        return jgg.compileJobGraph(op);
-    }
 }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
deleted file mode 100644
index 9532b1036d2cb..0000000000000
--- a/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
+++ /dev/null
@@ -1,95 +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.actions;
-
-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.test.util.MultipleProgramsTestBaseJUnit4;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-/** Tests the methods that bring elements back to the client driver program. */
-@RunWith(Parameterized.class)
-public class CountCollectITCase extends MultipleProgramsTestBaseJUnit4 {
-
-    public CountCollectITCase(TestExecutionMode mode) {
-        super(mode);
-    }
-
-    @Test
-    public void testSimple() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Integer[] input = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
-
-        DataSet data = env.fromElements(input);
-
-        // count
-        long numEntries = data.count();
-        assertEquals(10, numEntries);
-
-        // collect
-        ArrayList list = (ArrayList) data.collect();
-        assertArrayEquals(input, list.toArray());
-    }
-
-    @Test
-    public void testAdvanced() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        env.getConfig().disableObjectReuse();
-
-        DataSet data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
-        DataSet data2 = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
-
-        DataSet> data3 = data.cross(data2);
-
-        // count
-        long numEntries = data3.count();
-        assertEquals(100, numEntries);
-
-        // collect
-        ArrayList> list =
-                (ArrayList>) data3.collect();
-
-        // set expected entries in a hash map to true
-        HashMap, Boolean> expected =
-                new HashMap, Boolean>();
-        for (int i = 1; i <= 10; i++) {
-            for (int j = 1; j <= 10; j++) {
-                expected.put(new Tuple2(i, j), true);
-            }
-        }
-
-        // check if all entries are contained in the hash map
-        for (int i = 0; i < 100; i++) {
-            Tuple2 element = list.get(i);
-            assertEquals(expected.get(element), true);
-            expected.remove(element);
-        }
-    }
-}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
deleted file mode 100644
index 8da1945a9d601..0000000000000
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
+++ /dev/null
@@ -1,180 +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.broadcastvars;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-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.operators.FlatMapOperator;
-import org.apache.flink.api.java.operators.JoinOperator;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.test.util.JavaProgramTestBaseJUnit4;
-import org.apache.flink.util.Collector;
-
-import java.util.Collection;
-import java.util.List;
-
-import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText;
-
-/** Test broadcast input after branching. */
-public class BroadcastBranchingITCase extends JavaProgramTestBaseJUnit4 {
-    private static final String RESULT = "(2,112)\n";
-
-    //              Sc1(id,a,b,c) --
-    //                              \
-    //    Sc2(id,x) --------         Jn2(id) -- Mp2 -- Sk
-    //                      \        /          / <=BC
-    //                       Jn1(id) -- Mp1 ----
-    //                      /
-    //    Sc3(id,y) --------
-    @Override
-    protected void testProgram() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        env.setParallelism(1);
-
-        // Sc1 generates M parameters a,b,c for second degree polynomials P(x) = ax^2 + bx + c
-        // identified by id
-        DataSet> sc1 =
-                env.fromElements(
-                        new Tuple4<>("1", 61, 6, 29),
-                        new Tuple4<>("2", 7, 13, 10),
-                        new Tuple4<>("3", 8, 13, 27));
-
-        // Sc2 generates N x values to be evaluated with the polynomial identified by id
-        DataSet> sc2 =
-                env.fromElements(new Tuple2<>("1", 5), new Tuple2<>("2", 3), new Tuple2<>("3", 6));
-
-        // Sc3 generates N y values to be evaluated with the polynomial identified by id
-        DataSet> sc3 =
-                env.fromElements(new Tuple2<>("1", 2), new Tuple2<>("2", 3), new Tuple2<>("3", 7));
-
-        // Jn1 matches x and y values on id and emits (id, x, y) triples
-        JoinOperator<
-                        Tuple2,
-                        Tuple2,
-                        Tuple3>
-                jn1 = sc2.join(sc3).where(0).equalTo(0).with(new Jn1());
-
-        // Jn2 matches polynomial and arguments by id, computes p = min(P(x),P(y)) and emits (id, p)
-        // tuples
-        JoinOperator<
-                        Tuple3,
-                        Tuple4,
-                        Tuple2>
-                jn2 = jn1.join(sc1).where(0).equalTo(0).with(new Jn2());
-
-        // Mp1 selects (id, x, y) triples where x = y and broadcasts z (=x=y) to Mp2
-        FlatMapOperator, Tuple2> mp1 =
-                jn1.flatMap(new Mp1());
-
-        // Mp2 filters out all p values which can be divided by z
-        List> result =
-                jn2.flatMap(new Mp2()).withBroadcastSet(mp1, "z").collect();
-
-        compareResultAsText(result, RESULT);
-    }
-
-    private static class Jn1
-            implements JoinFunction<
-                    Tuple2,
-                    Tuple2,
-                    Tuple3> {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public Tuple3 join(
-                Tuple2 first, Tuple2 second) throws Exception {
-            return new Tuple3<>(first.f0, first.f1, second.f1);
-        }
-    }
-
-    private static class Jn2
-            implements JoinFunction<
-                    Tuple3,
-                    Tuple4,
-                    Tuple2> {
-        private static final long serialVersionUID = 1L;
-
-        private static int p(int x, int a, int b, int c) {
-            return a * x * x + b * x + c;
-        }
-
-        @Override
-        public Tuple2 join(
-                Tuple3 first,
-                Tuple4 second)
-                throws Exception {
-            int x = first.f1;
-            int y = first.f2;
-            int a = second.f1;
-            int b = second.f2;
-            int c = second.f3;
-
-            int pX = p(x, a, b, c);
-            int pY = p(y, a, b, c);
-            int min = Math.min(pX, pY);
-            return new Tuple2<>(first.f0, min);
-        }
-    }
-
-    private static class Mp1
-            implements FlatMapFunction, Tuple2> {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public void flatMap(
-                Tuple3 value, Collector> out)
-                throws Exception {
-            if (value.f1.compareTo(value.f2) == 0) {
-                out.collect(new Tuple2<>(value.f0, value.f1));
-            }
-        }
-    }
-
-    private static class Mp2
-            extends RichFlatMapFunction, Tuple2> {
-        private static final long serialVersionUID = 1L;
-
-        private Collection> zs;
-
-        @Override
-        public void open(OpenContext openContext) throws Exception {
-            this.zs = getRuntimeContext().getBroadcastVariable("z");
-        }
-
-        @Override
-        public void flatMap(Tuple2 value, Collector> out)
-                throws Exception {
-            int p = value.f1;
-
-            for (Tuple2 z : zs) {
-                if (z.f0.equals(value.f0)) {
-                    if (p % z.f1 != 0) {
-                        out.collect(value);
-                    }
-                }
-            }
-        }
-    }
-}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java
deleted file mode 100644
index 1e331abc07805..0000000000000
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java
+++ /dev/null
@@ -1,78 +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.broadcastvars;
-
-import org.apache.flink.api.common.functions.OpenContext;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.test.util.JavaProgramTestBaseJUnit4;
-
-import org.junit.Assert;
-
-import java.util.List;
-
-/** Test broadcast input after union. */
-public class BroadcastUnionITCase extends JavaProgramTestBaseJUnit4 {
-    private static final String BC_NAME = "bc";
-
-    @Override
-    protected void testProgram() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        env.setParallelism(4);
-
-        DataSet input = env.generateSequence(1, 10);
-        DataSet bc1 = env.generateSequence(1, 5);
-        DataSet bc2 = env.generateSequence(6, 10);
-
-        List result =
-                input.map(new Mapper())
-                        .withBroadcastSet(bc1.union(bc2), BC_NAME)
-                        .reduce(new Reducer())
-                        .collect();
-
-        Assert.assertEquals(Long.valueOf(3025), result.get(0));
-    }
-
-    private static class Mapper extends RichMapFunction {
-        private List values;
-
-        @Override
-        public void open(OpenContext openContext) {
-            values = getRuntimeContext().getBroadcastVariable(BC_NAME);
-        }
-
-        @Override
-        public Long map(Long value) throws Exception {
-            long sum = 0;
-            for (Long v : values) {
-                sum += value * v;
-            }
-            return sum;
-        }
-    }
-
-    private static class Reducer implements ReduceFunction {
-        @Override
-        public Long reduce(Long value1, Long value2) throws Exception {
-            return value1 + value2;
-        }
-    }
-}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
deleted file mode 100644
index 72b4cdb5b9dbd..0000000000000
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
+++ /dev/null
@@ -1,100 +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.broadcastvars;
-
-import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
-import org.apache.flink.api.common.functions.OpenContext;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.test.util.JavaProgramTestBaseJUnit4;
-import org.apache.flink.util.Collector;
-
-import org.junit.Assert;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-/** Test {@link BroadcastVariableInitializer}. */
-@SuppressWarnings("serial")
-public class BroadcastVarInitializationITCase extends JavaProgramTestBaseJUnit4 {
-
-    @Override
-    protected void testProgram() throws Exception {
-
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        env.setParallelism(4);
-
-        DataSet data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8);
-
-        IterativeDataSet iteration = data.iterate(10);
-
-        DataSet result =
-                data.reduceGroup(new PickOneAllReduce()).withBroadcastSet(iteration, "bc");
-
-        final List resultList = new ArrayList();
-        iteration.closeWith(result).output(new LocalCollectionOutputFormat(resultList));
-
-        env.execute();
-
-        Assert.assertEquals(8, resultList.get(0).intValue());
-    }
-
-    private static class PickOneAllReduce extends RichGroupReduceFunction {
-
-        private Integer bcValue;
-
-        @Override
-        public void open(OpenContext openContext) {
-            this.bcValue =
-                    getRuntimeContext()
-                            .getBroadcastVariableWithInitializer("bc", new PickFirstInitializer());
-        }
-
-        @Override
-        public void reduce(Iterable records, Collector out) {
-            if (bcValue == null) {
-                return;
-            }
-            final int x = bcValue;
-
-            for (Integer y : records) {
-                if (y > x) {
-                    out.collect(y);
-                    return;
-                }
-            }
-
-            out.collect(bcValue);
-        }
-    }
-
-    private static class PickFirstInitializer
-            implements BroadcastVariableInitializer {
-
-        @Override
-        public Integer initializeBroadcastVariable(Iterable data) {
-            Iterator iter = data.iterator();
-            return iter.hasNext() ? iter.next() : null;
-        }
-    }
-}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
index 7622241d7041a..f1aeb7f41928a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
@@ -20,17 +20,12 @@
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobStatus;
-import org.apache.flink.api.common.Plan;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.configuration.RpcOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
 import org.apache.flink.test.util.MiniClusterWithClientResource;
@@ -92,11 +87,10 @@ private static Configuration getConfiguration() {
 
     // --------------------------------------------------------------------------------------------
 
-    protected void runAndCancelJob(Plan plan, final int msecsTillCanceling, int maxTimeTillCanceled)
+    protected void runAndCancelJob(
+            JobGraph jobGraph, final int msecsTillCanceling, int maxTimeTillCanceled)
             throws Exception {
         // submit job
-        final JobGraph jobGraph = getJobGraph(plan);
-
         final long rpcTimeout = configuration.get(RpcOptions.ASK_TIMEOUT_DURATION).toMillis();
 
         ClusterClient client = CLUSTER.getClusterClient();
@@ -129,11 +123,4 @@ protected void runAndCancelJob(Plan plan, final int msecsTillCanceling, int maxT
         }
         assertEquals(JobStatus.CANCELED, jobStatusAfterCancel);
     }
-
-    private JobGraph getJobGraph(final Plan plan) {
-        final Optimizer pc = new Optimizer(new DataStatistics(), getConfiguration());
-        final OptimizedPlan op = pc.compile(plan);
-        final JobGraphGenerator jgg = new JobGraphGenerator();
-        return jgg.compileJobGraph(op);
-    }
 }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
index 057132d41a395..90c9231ab8db5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
@@ -21,20 +21,17 @@
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.functions.RichJoinFunction;
-import org.apache.flink.api.common.io.GenericInputFormat;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
-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.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.test.util.InfiniteIntegerTupleInputFormat;
 import org.apache.flink.test.util.UniformIntTupleGeneratorInputFormat;
 
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-
 /** Test job cancellation from within a JoinFunction. */
 @Ignore("Takes too long.")
 public class JoinCancelingITCase extends CancelingTestBase {
@@ -61,35 +58,36 @@ private void executeTask(
             boolean slow,
             int parallelism)
             throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        DataSet> input1 =
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        DataStreamSource> input1 =
                 env.createInput(new InfiniteIntegerTupleInputFormat(slow));
-        DataSet> input2 =
+        DataStreamSource> input2 =
                 env.createInput(new InfiniteIntegerTupleInputFormat(slow));
 
-        input1.join(input2, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE)
-                .where(0)
-                .equalTo(0)
-                .with(joiner)
-                .output(new DiscardingOutputFormat>());
+        input1.join(input2)
+                .where(value -> value.f0)
+                .equalTo(value -> value.f0)
+                .window(GlobalWindows.createWithEndOfStreamTrigger())
+                .apply(joiner)
+                .sinkTo(new DiscardingSink<>());
 
         env.setParallelism(parallelism);
 
-        runAndCancelJob(env.createProgramPlan(), 5 * 1000, 10 * 1000);
+        runAndCancelJob(env.getStreamGraph().getJobGraph(), 5 * 1000, 10 * 1000);
     }
 
     @Test
-    public void testCancelSortMatchWhileReadingSlowInputs() throws Exception {
+    public void testCancelWhileReadingSlowInputs() throws Exception {
         executeTask(new SimpleMatcher(), true);
     }
 
     @Test
-    public void testCancelSortMatchWhileReadingFastInputs() throws Exception {
+    public void testCancelWhileReadingFastInputs() throws Exception {
         executeTask(new SimpleMatcher(), false);
     }
 
     @Test
-    public void testCancelSortMatchPriorToFirstRecordReading() throws Exception {
+    public void testCancelPriorToFirstRecordReading() throws Exception {
         executeTask(new StuckInOpenMatcher(), false);
     }
 
@@ -104,62 +102,33 @@ private void executeTaskWithGenerator(
             int msecsTillCanceling,
             int maxTimeTillCanceled)
             throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        DataSet> input1 =
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        DataStreamSource> input1 =
                 env.createInput(new UniformIntTupleGeneratorInputFormat(keys, vals));
-        DataSet> input2 =
+        DataStreamSource> input2 =
                 env.createInput(new UniformIntTupleGeneratorInputFormat(keys, vals));
 
-        input1.join(input2, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE)
-                .where(0)
-                .equalTo(0)
-                .with(joiner)
-                .output(new DiscardingOutputFormat>());
-
-        env.setParallelism(PARALLELISM);
+        input1.join(input2)
+                .where(value -> value.f0)
+                .equalTo(value -> value.f0)
+                .window(GlobalWindows.createWithEndOfStreamTrigger())
+                .apply(joiner)
+                .sinkTo(new DiscardingSink<>());
 
-        runAndCancelJob(env.createProgramPlan(), msecsTillCanceling, maxTimeTillCanceled);
-    }
-
-    @Test
-    public void testCancelSortMatchWhileDoingHeavySorting() throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        HeavyCompareGeneratorInputFormat input = new HeavyCompareGeneratorInputFormat(100);
-        DataSet> input1 = env.createInput(input);
-        DataSet> input2 = env.createInput(input);
-
-        input1.join(input2, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE)
-                .where(0)
-                .equalTo(0)
-                .with(
-                        new JoinFunction<
-                                Tuple2,
-                                Tuple2,
-                                Tuple2>() {
-                            @Override
-                            public Tuple2 join(
-                                    Tuple2 first,
-                                    Tuple2 second)
-                                    throws Exception {
-                                throw new Exception(
-                                        "Job should be canceled in sort-merge phase, never run here ...");
-                            }
-                        })
-                .output(new DiscardingOutputFormat>());
-
-        runAndCancelJob(env.createProgramPlan(), 30 * 1000, 60 * 1000);
+        runAndCancelJob(
+                env.getStreamGraph().getJobGraph(), msecsTillCanceling, maxTimeTillCanceled);
     }
 
     // --------------- Test Sort Matches that are canceled while in the Matching Phase
     // -----------------
 
     @Test
-    public void testCancelSortMatchWhileJoining() throws Exception {
+    public void testCancelWhileJoining() throws Exception {
         executeTaskWithGenerator(new DelayingMatcher(), 500, 3, 10 * 1000, 20 * 1000);
     }
 
     @Test
-    public void testCancelSortMatchWithLongCancellingResponse() throws Exception {
+    public void testCancelWithLongCancellingResponse() throws Exception {
         executeTaskWithGenerator(
                 new LongCancelTimeMatcher(), 500, 3, 10 * 1000, 10 * 1000);
     }
@@ -168,7 +137,7 @@ public void testCancelSortMatchWithLongCancellingResponse() throws Exception {
     // ---------------------------------
 
     @Test
-    public void testCancelSortMatchWithHighparallelism() throws Exception {
+    public void testCancelWithHighparallelism() throws Exception {
         executeTask(new SimpleMatcher(), false, 64);
     }
 
@@ -234,34 +203,3 @@ public Tuple2 join(Tuple2 first, Tuple2 second) throws E
         }
     }
 }
-
-class HeavyCompare implements Comparable, java.io.Serializable {
-    @Override
-    public int compareTo(org.apache.flink.test.cancelling.HeavyCompare o) {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException iex) {
-        }
-        return 0;
-    }
-}
-
-class HeavyCompareGeneratorInputFormat extends GenericInputFormat> {
-    private int valueTotal;
-
-    public HeavyCompareGeneratorInputFormat(int numVals) {
-        this.valueTotal = numVals;
-    }
-
-    @Override
-    public boolean reachedEnd() throws IOException {
-        return valueTotal <= 0;
-    }
-
-    @Override
-    public Tuple2 nextRecord(Tuple2 reuse)
-            throws IOException {
-        valueTotal -= 1;
-        return new Tuple2<>(new HeavyCompare(), 20110701);
-    }
-}
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
index 70ab671873217..87cd8e25bb659 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
@@ -21,8 +21,8 @@
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
 import org.apache.flink.test.util.InfiniteIntegerInputFormat;
 
 import org.junit.Test;
@@ -51,15 +51,15 @@ public void testMapPriorToFirstRecordReading() throws Exception {
     }
 
     public void executeTask(MapFunction mapper) throws Exception {
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
         env.createInput(new InfiniteIntegerInputFormat(false))
                 .map(mapper)
-                .output(new DiscardingOutputFormat());
+                .sinkTo(new DiscardingSink<>());
 
         env.setParallelism(PARALLELISM);
 
-        runAndCancelJob(env.createProgramPlan(), 5 * 1000, 10 * 1000);
+        runAndCancelJob(env.getStreamGraph().getJobGraph(), 5 * 1000, 10 * 1000);
     }
 
     // --------------------------------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
index deea6632375ba..857847f566c98 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
@@ -22,7 +22,6 @@
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.io.FilePathFilter;
 import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.api.java.io.TextInputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.runtime.execution.SuppressRestartsException;
@@ -32,6 +31,7 @@
 import org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
 import org.apache.flink.streaming.api.functions.source.legacy.ContinuousFileMonitoringFunction;
+import org.apache.flink.streaming.api.legacy.io.TextInputFormat;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index ed7eb48b8ab84..10e06cacc3931 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -38,9 +38,7 @@
 import org.apache.flink.runtime.testutils.MiniClusterResource;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.testdata.KMeansData;
 import org.apache.flink.test.util.SuccessException;
-import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.SerializedThrowable;
 import org.apache.flink.util.TestLogger;
@@ -56,7 +54,6 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URL;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.concurrent.TimeUnit;
@@ -159,24 +156,6 @@ public static void tearDownClass() {
     @After
     public void tearDown() {
         TestStreamEnvironment.unsetAsContext();
-        TestEnvironment.unsetAsContext();
-    }
-
-    @Test
-    public void testCustomSplitJobWithCustomClassLoaderJar() throws ProgramInvocationException {
-
-        PackagedProgram inputSplitTestProg =
-                PackagedProgram.newBuilder()
-                        .setJarFile(new File(INPUT_SPLITS_PROG_JAR_FILE))
-                        .build();
-
-        TestEnvironment.setAsContext(
-                miniClusterResource.getMiniCluster(),
-                parallelism,
-                Collections.singleton(new Path(INPUT_SPLITS_PROG_JAR_FILE)),
-                Collections.emptyList());
-
-        inputSplitTestProg.invokeInteractiveModeForExecution();
     }
 
     @Test
@@ -196,24 +175,6 @@ public void testStreamingCustomSplitJobWithCustomClassLoader()
         streamingInputSplitTestProg.invokeInteractiveModeForExecution();
     }
 
-    @Test
-    public void testCustomSplitJobWithCustomClassLoaderPath()
-            throws IOException, ProgramInvocationException {
-        URL classpath = new File(INPUT_SPLITS_PROG_JAR_FILE).toURI().toURL();
-        PackagedProgram inputSplitTestProg2 =
-                PackagedProgram.newBuilder()
-                        .setJarFile(new File(INPUT_SPLITS_PROG_JAR_FILE))
-                        .build();
-
-        TestEnvironment.setAsContext(
-                miniClusterResource.getMiniCluster(),
-                parallelism,
-                Collections.emptyList(),
-                Collections.singleton(classpath));
-
-        inputSplitTestProg2.invokeInteractiveModeForExecution();
-    }
-
     @Test
     public void testStreamingClassloaderJobWithCustomClassLoader()
             throws ProgramInvocationException {
@@ -264,32 +225,12 @@ public void testCheckpointedStreamingClassloaderJobWithCustomClassLoader()
                                 "org.apache.flink.test.classloading.jar.CheckpointedStreamingProgram$SuccessException"));
     }
 
-    @Test
-    public void testKMeansJobWithCustomClassLoader() throws ProgramInvocationException {
-        PackagedProgram kMeansProg =
-                PackagedProgram.newBuilder()
-                        .setJarFile(new File(KMEANS_JAR_PATH))
-                        .setArguments(
-                                new String[] {
-                                    KMeansData.DATAPOINTS, KMeansData.INITIAL_CENTERS, "25"
-                                })
-                        .build();
-
-        TestEnvironment.setAsContext(
-                miniClusterResource.getMiniCluster(),
-                parallelism,
-                Collections.singleton(new Path(KMEANS_JAR_PATH)),
-                Collections.emptyList());
-
-        kMeansProg.invokeInteractiveModeForExecution();
-    }
-
     @Test
     public void testUserCodeTypeJobWithCustomClassLoader() throws ProgramInvocationException {
         PackagedProgram userCodeTypeProg =
                 PackagedProgram.newBuilder().setJarFile(new File(USERCODETYPE_JAR_PATH)).build();
 
-        TestEnvironment.setAsContext(
+        TestStreamEnvironment.setAsContext(
                 miniClusterResource.getMiniCluster(),
                 parallelism,
                 Collections.singleton(new Path(USERCODETYPE_JAR_PATH)),
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
index fc94a67fa9dba..721f7078d8eae 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
@@ -22,6 +22,7 @@
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
 import org.apache.flink.api.common.state.CheckpointListener;
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
@@ -31,7 +32,8 @@
 import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
@@ -86,7 +88,10 @@ public Integer getKey(Tuple2 value) throws Exception {
                             }
                         })
                 .flatMap(new ReducingStateFlatMap())
-                .writeAsText(outputPath, FileSystem.WriteMode.OVERWRITE);
+                .sinkTo(
+                        FileSink.forRowFormat(
+                                        new Path(outputPath), new SimpleStringEncoder())
+                                .build());
 
         env.execute();
     }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
index 721c43fb6a1b7..18c42c508aa32 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
@@ -23,14 +23,14 @@
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-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.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
+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 java.util.ArrayList;
 import java.util.Arrays;
@@ -42,9 +42,9 @@ public class CustomInputSplitProgram {
 
     public static void main(String[] args) throws Exception {
 
-        ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-        DataSet data = env.createInput(new CustomInputFormat());
+        DataStream data = env.createInput(new CustomInputFormat());
 
         data.map(
                         new MapFunction>() {
@@ -53,7 +53,7 @@ public Tuple2 map(Integer value) {
                                 return new Tuple2(value, value * 0.5);
                             }
                         })
-                .output(new DiscardingOutputFormat>());
+                .sinkTo(new DiscardingSink<>());
 
         env.execute();
     }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
index 99c89d1da5a6a..016c7134767b4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
@@ -22,10 +22,13 @@
 import org.apache.flink.api.common.functions.OpenContext;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.CheckpointStorageUtils;
@@ -80,7 +83,10 @@ public Integer getKey(Tuple2 value) throws Exception {
                             }
                         })
                 .flatMap(new ReducingStateFlatMap())
-                .writeAsText(outputPath);
+                .sinkTo(
+                        FileSink.forRowFormat(
+                                        new Path(outputPath), new SimpleStringEncoder())
+                                .build());
 
         env.execute();
     }
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
deleted file mode 100644
index d601421de7040..0000000000000
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
+++ /dev/null
@@ -1,307 +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.classloading.jar;
-
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.accumulators.SimpleAccumulator;
-import org.apache.flink.api.common.functions.OpenContext;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-import java.util.Collection;
-
-/**
- * This class belongs to the {@link org.apache.flink.test.classloading.ClassLoaderITCase} test.
- *
- * 

It tests dynamic class loading for: - * - *

    - *
  • Custom Functions - *
  • Custom Data Types - *
  • Custom Accumulators - *
  • Custom Types in collect() - *
- * - *

It's removed by Maven from classpath, so other tests must not depend on it. - */ -@SuppressWarnings("serial") -public class KMeansForTest { - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(String[] args) throws Exception { - if (args.length < 3) { - throw new IllegalArgumentException("Missing parameters"); - } - - final String pointsData = args[0]; - final String centersData = args[1]; - final int numIterations = Integer.parseInt(args[2]); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet points = - env.fromElements(pointsData.split("\n")).map(new TuplePointConverter()); - - DataSet centroids = - env.fromElements(centersData.split("\n")).map(new TupleCentroidConverter()); - - // set number of bulk iterations for KMeans algorithm - IterativeDataSet loop = centroids.iterate(numIterations); - - DataSet newCentroids = - points - // compute closest centroid for each point - .map(new SelectNearestCenter()) - .withBroadcastSet(loop, "centroids") - - // count and sum point coordinates for each centroid (test pojo return type) - .map(new CountAppender()) - - // !test if key expressions are working! - .groupBy("field0") - .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); - - // test that custom data type collects are working - finalCentroids.collect(); - } - - // ************************************************************************* - // DATA TYPES - // ************************************************************************* - - /** A simple two-dimensional point. */ - public static class Point { - - 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 - // ************************************************************************* - - /** Converts a {@code Tuple2} into a {@link Point}. */ - public static final class TuplePointConverter extends RichMapFunction { - - @Override - public Point map(String str) { - String[] fields = str.split("\\|"); - return new Point(Double.parseDouble(fields[1]), Double.parseDouble(fields[2])); - } - } - - /** Converts a {@code Tuple3} into a {@link Centroid}. */ - public static final class TupleCentroidConverter extends RichMapFunction { - - @Override - public Centroid map(String str) { - String[] fields = str.split("\\|"); - return new Centroid( - Integer.parseInt(fields[0]), - Double.parseDouble(fields[1]), - Double.parseDouble(fields[2])); - } - } - - /** Determines the closest cluster center for a data point. */ - public static final class SelectNearestCenter - extends RichMapFunction> { - - private Collection centroids; - private CustomAccumulator acc; - - /** Reads the centroid values from a broadcast variable into a collection. */ - @Override - public void open(OpenContext openContext) throws Exception { - this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); - this.acc = new CustomAccumulator(); - getRuntimeContext().addAccumulator("myAcc", this.acc); - } - - @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. - acc.add(1L); - return new Tuple2(closestCentroidId, p); - } - } - - /** Use this so that we can check whether POJOs and the POJO comparator also work. */ - public static final class DummyTuple3IntPointLong { - public Integer field0; - public Point field1; - public Long field2; - - public DummyTuple3IntPointLong() {} - - DummyTuple3IntPointLong(Integer f0, Point f1, Long f2) { - this.field0 = f0; - this.field1 = f1; - this.field2 = f2; - } - } - - /** Appends a count variable to the tuple. */ - public static final class CountAppender - extends RichMapFunction, DummyTuple3IntPointLong> { - - @Override - public DummyTuple3IntPointLong map(Tuple2 t) { - return new DummyTuple3IntPointLong(t.f0, t.f1, 1L); - } - } - - /** Sums and counts point coordinates. */ - public static final class CentroidAccumulator - extends RichReduceFunction { - - @Override - public DummyTuple3IntPointLong reduce( - DummyTuple3IntPointLong val1, DummyTuple3IntPointLong val2) { - return new DummyTuple3IntPointLong( - val1.field0, val1.field1.add(val2.field1), val1.field2 + val2.field2); - } - } - - /** Computes new centroid from coordinate sum and count of points. */ - public static final class CentroidAverager - extends RichMapFunction { - - @Override - public Centroid map(DummyTuple3IntPointLong value) { - return new Centroid(value.field0, value.field1.div(value.field2)); - } - } - - private static class CustomAccumulator implements SimpleAccumulator { - - private long value; - - @Override - public void add(Long value) { - this.value += value; - } - - @Override - public Long getLocalValue() { - return this.value; - } - - @Override - public void resetLocal() { - this.value = 0L; - } - - @Override - public void merge(Accumulator other) { - this.value += other.getLocalValue(); - } - - @Override - public Accumulator clone() { - CustomAccumulator acc = new CustomAccumulator(); - acc.value = this.value; - return acc; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java index e5bc5ce2dd40d..d807f8baecd34 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java @@ -19,9 +19,10 @@ package org.apache.flink.test.classloading.jar; 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.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; /** * Test class used by the {@link org.apache.flink.test.classloading.ClassLoaderITCase}. @@ -43,11 +44,11 @@ public String toString() { } public static void main(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet input = env.fromElements(1, 2, 3, 4, 5); + DataStreamSource input = env.fromData(1, 2, 3, 4, 5); - DataSet customTypes = + DataStream customTypes = input.map( new MapFunction() { private static final long serialVersionUID = @@ -60,7 +61,7 @@ public CustomType map(Integer integer) throws Exception { }) .rebalance(); - DataSet result = + DataStream result = customTypes.map( new MapFunction() { private static final long serialVersionUID = -7950126399899584991L; @@ -71,7 +72,7 @@ public Integer map(CustomType value) throws Exception { } }); - result.output(new DiscardingOutputFormat()); + result.sinkTo(new DiscardingSink<>()); env.execute(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java b/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java index 29e15a2a5e9a6..0c5bc1aed2186 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java @@ -19,11 +19,14 @@ package org.apache.flink.test.distributedcache; import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; +import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.AbstractTestBaseJUnit4; import org.apache.flink.util.Collector; @@ -81,18 +84,14 @@ public void testStreamingDistributedCache() throws Exception { String textPath = createTempFile("count.txt", DATA); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.registerCachedFile(textPath, "cache_test"); - env.readTextFile(textPath).flatMap(new WordChecker()); + FileSource source = + FileSource.forRecordStreamFormat(new TextLineInputFormat(), new Path(textPath)) + .build(); + env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source") + .flatMap(new WordChecker()); env.execute(); } - @Test - public void testBatchDistributedCache() throws Exception { - String textPath = createTempFile("count.txt", DATA); - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.registerCachedFile(textPath, "cache_test"); - env.readTextFile(textPath).flatMap(new WordChecker()).count(); - } - private static class WordChecker extends RichFlatMapFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java index 6713659f09a52..7cfab00d5e393 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java @@ -19,16 +19,19 @@ package org.apache.flink.test.example.client; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.client.deployment.executors.LocalExecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; +import org.apache.flink.streaming.api.legacy.io.TextOutputFormat; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.testfunctions.Tokenizer; import org.apache.flink.util.TestLogger; @@ -80,10 +83,10 @@ public void testLocalExecutorWithWordCount() throws InterruptedException { config.set(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); config.set(DeploymentOptions.ATTACHED, true); - Plan wcPlan = getWordCountPlan(inFile, outFile, parallelism); - wcPlan.setExecutionConfig(new ExecutionConfig()); + StreamGraph wcStreamGraph = getWordCountStreamGraph(inFile, outFile, parallelism); JobClient jobClient = - executor.execute(wcPlan, config, ClassLoader.getSystemClassLoader()).get(); + executor.execute(wcStreamGraph, config, ClassLoader.getSystemClassLoader()) + .get(); jobClient.getJobExecutionResult().get(); } catch (Exception e) { e.printStackTrace(); @@ -95,8 +98,7 @@ public void testLocalExecutorWithWordCount() throws InterruptedException { @Test(timeout = 60_000) public void testMiniClusterShutdownOnErrors() throws Exception { - Plan runtimeExceptionPlan = getRuntimeExceptionPlan(); - runtimeExceptionPlan.setExecutionConfig(new ExecutionConfig()); + StreamGraph runtimeExceptionPlan = getRuntimeExceptionPlan(); Configuration config = new Configuration(); config.set(DeploymentOptions.ATTACHED, true); @@ -113,20 +115,22 @@ public void testMiniClusterShutdownOnErrors() throws Exception { assertThat(miniCluster.isRunning(), is(false)); } - private Plan getWordCountPlan(File inFile, File outFile, int parallelism) { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + private StreamGraph getWordCountStreamGraph(File inFile, File outFile, int parallelism) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(parallelism); - env.readTextFile(inFile.getAbsolutePath()) + env.createInput(new TextInputFormat(new Path(inFile.getAbsolutePath()))) .flatMap(new Tokenizer()) - .groupBy(0) + .keyBy(x -> x.f0) .sum(1) - .writeAsCsv(outFile.getAbsolutePath()); - return env.createProgramPlan(); + .addSink( + new OutputFormatSinkFunction<>( + new TextOutputFormat<>(new Path(outFile.getAbsolutePath())))); + return env.getStreamGraph(); } - private Plan getRuntimeExceptionPlan() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements(1) + private StreamGraph getRuntimeExceptionPlan() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.fromData(1) .map( element -> { if (element == 1) { @@ -134,7 +138,7 @@ private Plan getRuntimeExceptionPlan() { } return element; }) - .output(new DiscardingOutputFormat<>()); - return env.createProgramPlan(); + .sinkTo(new DiscardingSink<>()); + return env.getStreamGraph(); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java index e40c911f80fca..b0baa67abd1f7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java @@ -21,13 +21,11 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestartStrategyOptions; -import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import org.junit.Assert; +import org.apache.flink.util.CollectionUtil; import java.time.Duration; import java.util.List; @@ -50,16 +48,7 @@ protected void testProgram() throws Exception { // test failing version try { executeTask(new FailingTestMapper(), 1); - } catch (RuntimeException e) { // expected for collection execution - if (!isCollectionExecution()) { - Assert.fail(); - } - // for collection execution, no restarts. So, exception should be appended with 0. - assertTrue(findThrowableWithMessage(e, EXCEPTION_STRING + ":0").isPresent()); - } catch (JobExecutionException e) { // expected for cluster execution - if (isCollectionExecution()) { - Assert.fail(); - } + } catch (RuntimeException e) { // expected for cluster execution // for cluster execution, one restart. So, exception should be appended with 1. assertTrue(findThrowableWithMessage(e, EXCEPTION_STRING + ":1").isPresent()); } @@ -73,12 +62,12 @@ private void executeTask(MapFunction mapper, int retries) throws Exc configuration.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, retries); configuration.set( RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofMillis(0)); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setParallelism(1); - env.configure(configuration, Thread.currentThread().getContextClassLoader()); - - List result = env.generateSequence(1, 9).map(mapper).collect(); + List result = + CollectionUtil.iteratorToList( + env.fromSequence(1, 9).map(mapper).executeAndCollect()); compareResultAsText(result, "1\n2\n3\n4\n5\n6\n7\n8\n9"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java deleted file mode 100644 index 7d55c388c91be..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java +++ /dev/null @@ -1,72 +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.example.java; - -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.io.BufferedReader; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** Test for {@link ConnectedComponents}. */ -public class ConnectedComponentsITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - private String verticesPath; - private String edgesPath; - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - verticesPath = - createTempFile( - "vertices.txt", - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void testProgram() throws Exception { - ConnectedComponents.main( - "--vertices", verticesPath, - "--edges", edgesPath, - "--output", resultPath, - "--iterations", "100"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - ConnectedComponentsData.checkOddEvenResult(reader); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java deleted file mode 100644 index c997b81f83bc2..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java +++ /dev/null @@ -1,53 +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.example.java; - -import org.apache.flink.examples.java.graph.EnumTriangles; -import org.apache.flink.test.testdata.EnumTriangleData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Test {@link EnumTriangles}. */ -public class EnumTriangleBasicITCase extends JavaProgramTestBaseJUnit4 { - - protected String edgePath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - edgePath = createTempFile("edges", EnumTriangleData.EDGES); - resultPath = getTempDirPath("triangles"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_ID, resultPath); - } - - @Override - protected void testProgram() throws Exception { - EnumTriangles.main( - new String[] { - "--edges", edgePath, - "--output", resultPath - }); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java deleted file mode 100644 index 96e54fd3d8032..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java +++ /dev/null @@ -1,101 +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.example.java; - -import org.apache.flink.examples.java.graph.PageRank; -import org.apache.flink.test.testdata.PageRankData; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.FileUtils; - -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; -import java.util.UUID; - -import static org.apache.flink.test.util.TestBaseUtils.compareKeyValuePairsWithDelta; - -/** Test for {@link PageRank}. */ -@RunWith(Parameterized.class) -public class PageRankITCase extends MultipleProgramsTestBaseJUnit4 { - - public PageRankITCase(TestExecutionMode mode) { - super(mode); - } - - private String verticesPath; - private String edgesPath; - private String resultPath; - private String expected; - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception { - final File folder = tempFolder.newFolder(); - final File resultFile = new File(folder, UUID.randomUUID().toString()); - resultPath = resultFile.toURI().toString(); - - File verticesFile = tempFolder.newFile(); - FileUtils.writeFileUtf8(verticesFile, PageRankData.VERTICES); - - File edgesFile = tempFolder.newFile(); - FileUtils.writeFileUtf8(edgesFile, PageRankData.EDGES); - - verticesPath = verticesFile.toURI().toString(); - edgesPath = edgesFile.toURI().toString(); - } - - @After - public void after() throws Exception { - compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01); - } - - @Test - public void testPageRankSmallNumberOfIterations() throws Exception { - PageRank.main( - new String[] { - "--pages", verticesPath, - "--links", edgesPath, - "--output", resultPath, - "--numPages", PageRankData.NUM_VERTICES + "", - "--iterations", "3" - }); - expected = PageRankData.RANKS_AFTER_3_ITERATIONS; - } - - @Test - public void testPageRankWithConvergenceCriterion() throws Exception { - PageRank.main( - new String[] { - "--pages", verticesPath, - "--links", edgesPath, - "--output", resultPath, - "--numPages", PageRankData.NUM_VERTICES + "", - "--vertices", "1000" - }); - expected = PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java deleted file mode 100644 index b542351b0f42b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java +++ /dev/null @@ -1,67 +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.example.java; - -import org.apache.flink.examples.java.graph.TransitiveClosureNaive; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.testdata.TransitiveClosureData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.io.BufferedReader; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** Test for {@link TransitiveClosureNaive}. */ -public class TransitiveClosureITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 100; - - private static final int NUM_EDGES = 500; - - private String edgesPath; - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void testProgram() throws Exception { - TransitiveClosureNaive.main( - "--edges", edgesPath, - "--output", resultPath, - "--iterations", "5"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - TransitiveClosureData.checkOddEvenResult(reader); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java deleted file mode 100644 index c6b27a1f281a3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java +++ /dev/null @@ -1,59 +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.example.java; - -import org.apache.flink.examples.java.relational.WebLogAnalysis; -import org.apache.flink.test.testdata.WebLogAnalysisData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Test for {@link WebLogAnalysis}. */ -public class WebLogAnalysisITCase extends JavaProgramTestBaseJUnit4 { - - private String docsPath; - private String ranksPath; - private String visitsPath; - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - docsPath = createTempFile("docs", WebLogAnalysisData.DOCS); - ranksPath = createTempFile("ranks", WebLogAnalysisData.RANKS); - visitsPath = createTempFile("visits", WebLogAnalysisData.VISITS); - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(WebLogAnalysisData.EXCEPTED_RESULT, resultPath); - } - - @Override - protected void testProgram() throws Exception { - WebLogAnalysis.main( - new String[] { - "--documents", docsPath, - "--ranks", ranksPath, - "--visits", visitsPath, - "--output", resultPath - }); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java index 7c3bed582431c..92a4b51088ea3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java @@ -19,7 +19,7 @@ package org.apache.flink.test.example.java; -import org.apache.flink.examples.java.wordcount.WordCount; +import org.apache.flink.streaming.examples.wordcount.WordCount; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; @@ -39,7 +39,7 @@ protected void preSubmit() throws Exception { @Override protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath); + compareResultsByLinesInMemory(WordCountData.COUNTS_AS_TUPLES, resultPath); } @Override @@ -47,7 +47,8 @@ protected void testProgram() throws Exception { WordCount.main( new String[] { "--input", textPath, - "--output", resultPath + "--output", resultPath, + "--execution-mode", "BATCH" }); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java index ed26f691e7bbd..5d79a4c10c807 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java @@ -19,11 +19,17 @@ package org.apache.flink.test.example.java; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.ReduceFunction; -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.Tuple3; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.apache.flink.util.Collector; @@ -53,12 +59,15 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet text = env.readTextFile(textPath); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); - DataSet counts = + DataStream text = env.createInput(new TextInputFormat(new Path(textPath))); + + DataStream counts = text.flatMap(new Tokenizer()) - .groupBy("complex.someTest") + .keyBy(x -> x.complex.someTest) + .window(GlobalWindows.createWithEndOfStreamTrigger()) .reduce( new ReduceFunction() { private static final long serialVersionUID = 1L; @@ -70,7 +79,8 @@ public WC reduce(WC value1, WC value2) { } }); - counts.writeAsText(resultPath); + counts.sinkTo( + FileSink.forRowFormat(new Path(resultPath), new SimpleStringEncoder()).build()); env.execute("WordCount with custom data types example"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java index edd96dbc7c436..934bffcc83a1e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java @@ -19,10 +19,16 @@ package org.apache.flink.test.example.java; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.ReduceFunction; -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.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.apache.flink.util.Collector; @@ -50,13 +56,15 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); - DataSet text = env.readTextFile(textPath); + DataStream text = env.createInput(new TextInputFormat(new Path(textPath))); - DataSet counts = + DataStream counts = text.flatMap(new Tokenizer()) - .groupBy("word") + .keyBy(x -> x.word) + .window(GlobalWindows.createWithEndOfStreamTrigger()) .reduce( new ReduceFunction() { private static final long serialVersionUID = 1L; @@ -66,7 +74,8 @@ public WC reduce(WC value1, WC value2) { } }); - counts.writeAsText(resultPath); + counts.sinkTo( + FileSink.forRowFormat(new Path(resultPath), new SimpleStringEncoder()).build()); env.execute("WordCount with custom data types example"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java index f377e2ea6ae14..f5a40ac3c167c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java @@ -22,8 +22,14 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.ReduceFunction; -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.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.apache.flink.util.Collector; @@ -53,12 +59,13 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet text = env.readTextFile(textPath); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStreamSource text = env.createInput(new TextInputFormat(new Path(textPath))); - DataSet counts = + DataStream counts = text.flatMap(new Tokenizer()) - .groupBy("word") + .keyBy(x -> x.word) + .window(GlobalWindows.createWithEndOfStreamTrigger()) .reduce( new ReduceFunction() { private static final long serialVersionUID = 1L; @@ -83,7 +90,9 @@ public WCBase map(WCBase value) throws Exception { } }); - counts.writeAsText(resultPath); + counts.sinkTo( + FileSink.forRowFormat(new Path(resultPath), new SimpleStringEncoder()) + .build()); env.execute("WordCount with custom data types example"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java index b55966aec9310..ebeef2d47b899 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java @@ -19,11 +19,17 @@ package org.apache.flink.test.example.java; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.ReduceFunction; -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.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.apache.flink.util.Collector; @@ -52,12 +58,15 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet text = env.readTextFile(textPath); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); - DataSet counts = + DataStream text = env.createInput(new TextInputFormat(new Path(textPath))); + + DataStream counts = text.flatMap(new Tokenizer()) - .groupBy("word") + .keyBy(x -> x.word) + .window(GlobalWindows.createWithEndOfStreamTrigger()) .reduce( new ReduceFunction() { private static final long serialVersionUID = 1L; @@ -79,7 +88,9 @@ public WCBase map(WCBase value) throws Exception { } }); - counts.writeAsText(resultPath); + counts.sinkTo( + FileSink.forRowFormat(new Path(resultPath), new SimpleStringEncoder()) + .build()); env.execute("WordCount with custom data types example"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java deleted file mode 100644 index 45cf0a0e89b27..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java +++ /dev/null @@ -1,72 +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.example.java; - -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.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.wordcount.WordCount; -import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import org.junit.Assert; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** WordCount with collection example. */ -public class WordCountWithCollectionITCase extends JavaProgramTestBaseJUnit4 { - - private final List> resultsCollected = - new ArrayList>(); - - @Override - protected void postSubmit() throws Exception { - String[] result = new String[resultsCollected.size()]; - for (int i = 0; i < result.length; i++) { - result[i] = resultsCollected.get(i).toString(); - } - Arrays.sort(result); - - String[] expected = WordCountData.COUNTS_AS_TUPLES.split("\n"); - Arrays.sort(expected); - - Assert.assertEquals( - "Different number of lines in expected and obtained result.", - expected.length, - result.length); - Assert.assertArrayEquals(expected, result); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet text = env.fromElements(WordCountData.TEXT); - DataSet> words = text.flatMap(new WordCount.Tokenizer()); - DataSet> result = words.groupBy(0).aggregate(Aggregations.SUM, 1); - - result.output(new LocalCollectionOutputFormat>(resultsCollected)); - env.execute("Word Count Collection"); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/execution/JobListenerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/execution/JobListenerITCase.java index a40ea64654b4b..8cf0edd05f36f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/execution/JobListenerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/execution/JobListenerITCase.java @@ -20,8 +20,6 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.client.deployment.executors.RemoteExecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; @@ -63,7 +61,7 @@ public void testExecuteCallsJobListenerOnBatchEnvironment() throws Exception { OneShotLatch submissionLatch = new OneShotLatch(); OneShotLatch executionLatch = new OneShotLatch(); - ExecutionEnvironment env = new ExecutionEnvironment(getClientConfiguration()); + StreamExecutionEnvironment env = new StreamExecutionEnvironment(getClientConfiguration()); env.registerJobListener( new JobListener() { @@ -80,7 +78,7 @@ public void onJobExecuted( } }); - env.fromElements(1, 2, 3, 4, 5).output(new DiscardingOutputFormat<>()); + env.fromData(1, 2, 3, 4, 5).sinkTo(new DiscardingSink<>()); JobExecutionResult jobExecutionResult = env.execute(); submissionLatch.await(2000L, TimeUnit.MILLISECONDS); @@ -94,7 +92,7 @@ public void testExecuteAsyncCallsJobListenerOnBatchEnvironment() throws Exceptio AtomicReference jobIdReference = new AtomicReference<>(); OneShotLatch submissionLatch = new OneShotLatch(); - ExecutionEnvironment env = new ExecutionEnvironment(getClientConfiguration()); + StreamExecutionEnvironment env = new StreamExecutionEnvironment(getClientConfiguration()); env.registerJobListener( new JobListener() { @@ -109,7 +107,7 @@ public void onJobExecuted( JobExecutionResult jobExecutionResult, Throwable throwable) {} }); - env.fromElements(1, 2, 3, 4, 5).output(new DiscardingOutputFormat<>()); + env.fromData(1, 2, 3, 4, 5).sinkTo(new DiscardingSink<>()); JobClient jobClient = env.executeAsync(); submissionLatch.await(2000L, TimeUnit.MILLISECONDS); @@ -122,7 +120,7 @@ public void onJobExecuted( public void testExecuteCallsJobListenerOnMainThreadOnBatchEnvironment() throws Exception { AtomicReference threadReference = new AtomicReference<>(); - ExecutionEnvironment env = new ExecutionEnvironment(getClientConfiguration()); + StreamExecutionEnvironment env = new StreamExecutionEnvironment(getClientConfiguration()); env.registerJobListener( new JobListener() { @@ -136,7 +134,7 @@ public void onJobExecuted( JobExecutionResult jobExecutionResult, Throwable throwable) {} }); - env.fromElements(1, 2, 3, 4, 5).output(new DiscardingOutputFormat<>()); + env.fromData(1, 2, 3, 4, 5).sinkTo(new DiscardingSink<>()); env.execute(); assertThat(Thread.currentThread(), is(threadReference.get())); @@ -146,7 +144,7 @@ public void onJobExecuted( public void testExecuteAsyncCallsJobListenerOnMainThreadOnBatchEnvironment() throws Exception { AtomicReference threadReference = new AtomicReference<>(); - ExecutionEnvironment env = new ExecutionEnvironment(getClientConfiguration()); + StreamExecutionEnvironment env = new StreamExecutionEnvironment(getClientConfiguration()); env.registerJobListener( new JobListener() { @@ -160,7 +158,7 @@ public void onJobExecuted( JobExecutionResult jobExecutionResult, Throwable throwable) {} }); - env.fromElements(1, 2, 3, 4, 5).output(new DiscardingOutputFormat<>()); + env.fromData(1, 2, 3, 4, 5).sinkTo(new DiscardingSink<>()); env.executeAsync(); assertThat(Thread.currentThread(), is(threadReference.get())); diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java deleted file mode 100644 index 2b2455dc58c90..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.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.test.io; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.types.BooleanValue; -import org.apache.flink.types.ByteValue; -import org.apache.flink.types.DoubleValue; -import org.apache.flink.types.FloatValue; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.ShortValue; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.FileUtils; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; -import java.util.List; -import java.util.Locale; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Tests for {@link ExecutionEnvironment#readCsvFile}. */ -@RunWith(Parameterized.class) -public class CsvReaderITCase extends MultipleProgramsTestBaseJUnit4 { - private String expected; - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - public CsvReaderITCase(TestExecutionMode mode) { - super(mode); - } - - private String createInputData(String data) throws Exception { - File file = tempFolder.newFile("input"); - FileUtils.writeFileUtf8(file, data); - - return file.toURI().toString(); - } - - @Test - public void testPOJOType() throws Exception { - final String inputData = "ABC,2.20,3\nDEF,5.1,5\nDEF,3.30,1\nGHI,3.30,10"; - final String dataPath = createInputData(inputData); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = - env.readCsvFile(dataPath).pojoType(POJOItem.class, new String[] {"f1", "f3", "f2"}); - List result = data.collect(); - - expected = "ABC,3,2.20\nDEF,5,5.10\nDEF,1,3.30\nGHI,10,3.30"; - compareResultAsText(result, expected); - } - - @Test - public void testPOJOTypeWithFieldsOrder() throws Exception { - final String inputData = "2.20,ABC,3\n5.1,DEF,5\n3.30,DEF,1\n3.30,GHI,10"; - final String dataPath = createInputData(inputData); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = - env.readCsvFile(dataPath).pojoType(POJOItem.class, new String[] {"f3", "f1", "f2"}); - List result = data.collect(); - - expected = "ABC,3,2.20\nDEF,5,5.10\nDEF,1,3.30\nGHI,10,3.30"; - compareResultAsText(result, expected); - } - - @Test(expected = IllegalArgumentException.class) - public void testPOJOTypeWithoutFieldsOrder() throws Exception { - final String inputData = ""; - final String dataPath = createInputData(inputData); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - env.readCsvFile(dataPath).pojoType(POJOItem.class); - } - - @Test(expected = IllegalArgumentException.class) - public void testPOJOTypeWitNullFieldsOrder() throws Exception { - final String inputData = ""; - final String dataPath = createInputData(inputData); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - env.readCsvFile(dataPath).pojoType(POJOItem.class, null); - } - - @Test - public void testPOJOTypeWithFieldsOrderAndFieldsSelection() throws Exception { - final String inputData = "3,2.20,ABC\n5,5.1,DEF\n1,3.30,DEF\n10,3.30,GHI"; - final String dataPath = createInputData(inputData); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = - env.readCsvFile(dataPath) - .includeFields(true, false, true) - .pojoType(POJOItem.class, new String[] {"f2", "f1"}); - List result = data.collect(); - - expected = "ABC,3,0.00\nDEF,5,0.00\nDEF,1,0.00\nGHI,10,0.00"; - compareResultAsText(result, expected); - } - - @Test - public void testValueTypes() throws Exception { - final String inputData = "ABC,true,1,2,3,4,5.0,6.0\nBCD,false,1,2,3,4,5.0,6.0"; - final String dataPath = createInputData(inputData); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet< - Tuple8< - StringValue, - BooleanValue, - ByteValue, - ShortValue, - IntValue, - LongValue, - FloatValue, - DoubleValue>> - data = - env.readCsvFile(dataPath) - .types( - StringValue.class, - BooleanValue.class, - ByteValue.class, - ShortValue.class, - IntValue.class, - LongValue.class, - FloatValue.class, - DoubleValue.class); - List< - Tuple8< - StringValue, - BooleanValue, - ByteValue, - ShortValue, - IntValue, - LongValue, - FloatValue, - DoubleValue>> - result = data.collect(); - - expected = inputData; - compareResultAsTuples(result, expected); - } - - /** POJO. */ - public static class POJOItem { - public String f1; - private int f2; - public double f3; - - public int getF2() { - return f2; - } - - public void setF2(int f2) { - this.f2 = f2; - } - - @Override - public String toString() { - return String.format(Locale.US, "%s,%d,%.02f", f1, f2, f3); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java index e7d68a4d4e211..124348ac5a4c8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java @@ -20,7 +20,8 @@ import org.apache.flink.api.common.operators.util.TestNonRichInputFormat; import org.apache.flink.api.common.operators.util.TestNonRichOutputFormat; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import static org.junit.Assert.fail; @@ -33,9 +34,10 @@ public class InputOutputITCase extends JavaProgramTestBaseJUnit4 { @Override protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); TestNonRichOutputFormat output = new TestNonRichOutputFormat(); - env.createInput(new TestNonRichInputFormat()).output(output); + env.createInput(new TestNonRichInputFormat()) + .addSink(new OutputFormatSinkFunction<>(output)); try { env.execute(); } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java index 698abdaeb64bd..7ba92963fe262 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java @@ -21,11 +21,12 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.RichOutputFormat; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileInputSplit; import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.legacy.io.TextInputFormat; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import java.io.IOException; @@ -54,8 +55,9 @@ protected void testProgram() throws Exception { readCalls = new ConcurrentLinkedQueue(); writeCalls = new ConcurrentLinkedQueue(); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.createInput(new TestInputFormat(new Path(inputPath))).output(new TestOutputFormat()); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.createInput(new TestInputFormat(new Path(inputPath))) + .addSink(new OutputFormatSinkFunction<>(new TestOutputFormat())); JobExecutionResult result = env.execute(); Object a = result.getAllAccumulatorResults().get("DATA_SOURCE_ACCUMULATOR"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java deleted file mode 100644 index 9a955393bed2e..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java +++ /dev/null @@ -1,86 +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.iterative; - -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Assert; - -import java.util.ArrayList; -import java.util.List; - -/** Integration test for a bulk iteration with an all reduce. */ -@SuppressWarnings("serial") -public class BulkIterationWithAllReducerITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8); - - IterativeDataSet iteration = data.iterate(10); - - DataSet result = - data.reduceGroup(new PickOneAllReduce()).withBroadcastSet(iteration, "bc"); - - final List resultList = new ArrayList(); - iteration.closeWith(result).output(new LocalCollectionOutputFormat(resultList)); - - env.execute(); - - Assert.assertEquals(8, resultList.get(0).intValue()); - } - - private static class PickOneAllReduce extends RichGroupReduceFunction { - - private Integer bcValue; - - @Override - public void open(OpenContext openContext) { - List bc = getRuntimeContext().getBroadcastVariable("bc"); - this.bcValue = bc.isEmpty() ? null : bc.get(0); - } - - @Override - public void reduce(Iterable records, Collector out) { - if (bcValue == null) { - return; - } - final int x = bcValue; - - for (Integer y : records) { - if (y > x) { - out.collect(y); - return; - } - } - - out.collect(bcValue); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java deleted file mode 100644 index 74d981d6dbc99..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java +++ /dev/null @@ -1,183 +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.iterative; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.JoinFunction; -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.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; -import org.apache.flink.api.java.operators.CoGroupOperator; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.JoinOperator; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import java.io.BufferedReader; -import java.util.Iterator; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** Delta iteration test implementing the connected components algorithm with a cogroup. */ -public class CoGroupConnectedComponentsITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - private static final int MAX_ITERATIONS = 100; - - protected String verticesPath; - protected String edgesPath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - verticesPath = - createTempFile( - "vertices.txt", - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - ConnectedComponentsData.checkOddEvenResult(reader); - } - } - - // -------------------------------------------------------------------------------------------- - // The test program - // -------------------------------------------------------------------------------------------- - - @Override - protected void testProgram() throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> initialVertices = - env.readCsvFile(verticesPath) - .fieldDelimiter(" ") - .types(Long.class) - .name("Vertices"); - - DataSet> edges = - env.readCsvFile(edgesPath) - .fieldDelimiter(" ") - .types(Long.class, Long.class) - .name("Edges"); - - DataSet> verticesWithId = - initialVertices - .map( - new MapFunction, Tuple2>() { - @Override - public Tuple2 map(Tuple1 value) - throws Exception { - return new Tuple2<>(value.f0, value.f0); - } - }) - .name("Assign Vertex Ids"); - - DeltaIteration, Tuple2> iteration = - verticesWithId.iterateDelta(verticesWithId, MAX_ITERATIONS, 0); - - JoinOperator, Tuple2, Tuple2> joinWithNeighbors = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with( - new JoinFunction< - Tuple2, - Tuple2, - Tuple2>() { - @Override - public Tuple2 join( - Tuple2 first, Tuple2 second) - throws Exception { - return new Tuple2<>(second.f1, first.f1); - } - }) - .name("Join Candidate Id With Neighbor"); - - CoGroupOperator, Tuple2, Tuple2> minAndUpdate = - joinWithNeighbors - .coGroup(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new MinIdAndUpdate()) - .name("min Id and Update"); - - iteration - .closeWith(minAndUpdate, minAndUpdate) - .writeAsCsv(resultPath, "\n", " ") - .name("Result"); - - env.execute("Workset Connected Components"); - } - - @ForwardedFieldsFirst("f1->f1") - @ForwardedFieldsSecond("f0->f0") - private static final class MinIdAndUpdate - implements CoGroupFunction, Tuple2, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception { - Iterator> current = second.iterator(); - if (!current.hasNext()) { - throw new Exception("Error: Id not encountered before."); - } - Tuple2 old = current.next(); - long oldId = old.f1; - - long minimumComponentID = Long.MAX_VALUE; - - for (Tuple2 candidate : first) { - long candidateComponentID = candidate.f1; - if (candidateComponentID < minimumComponentID) { - minimumComponentID = candidateComponentID; - } - } - - if (minimumComponentID < oldId) { - out.collect(new Tuple2<>(old.f0, minimumComponentID)); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java deleted file mode 100644 index 9d28601980fce..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java +++ /dev/null @@ -1,161 +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.iterative; - -import org.apache.flink.api.common.functions.RichCoGroupFunction; -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.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.graph.ConnectedComponents.DuplicateValue; -import org.apache.flink.examples.java.graph.ConnectedComponents.NeighborWithComponentIDJoin; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -/** Delta iteration test implementing the connected components algorithm with a cogroup. */ -@SuppressWarnings("serial") -public class CoGroupConnectedComponentsSecondITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - @Override - protected void testProgram() throws Exception { - - // set up execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // read vertex and edge data - DataSet vertices = - env.fromElements( - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES) - .split("\n")) - .map(new VertexParser()); - - DataSet> edges = - env.fromElements( - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED) - .split("\n")) - .flatMap(new EdgeParser()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); - - // apply the step logic: join with the edges, select the minimum neighbor, update if the - // component of the candidate is smaller - DataSet> changes = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .coGroup(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new MinIdAndUpdate()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(changes, changes); - - // emit result - List> resutTuples = new ArrayList<>(); - result.output(new LocalCollectionOutputFormat<>(resutTuples)); - - env.execute(); - } - - // -------------------------------------------------------------------------------------------- - // The test program - // -------------------------------------------------------------------------------------------- - - private static final class VertexParser extends RichMapFunction { - - @Override - public Long map(String value) throws Exception { - return Long.parseLong(value); - } - } - - private static final class EdgeParser extends RichFlatMapFunction> { - - @Override - public void flatMap(String value, Collector> out) throws Exception { - String[] parts = value.split(" "); - long v1 = Long.parseLong(parts[0]); - long v2 = Long.parseLong(parts[1]); - - out.collect(new Tuple2(v1, v2)); - out.collect(new Tuple2(v2, v1)); - } - } - - @ForwardedFieldsFirst("0") - @ForwardedFieldsSecond("0") - private static final class MinIdAndUpdate - extends RichCoGroupFunction< - Tuple2, Tuple2, Tuple2> { - - @Override - public void coGroup( - Iterable> candidates, - Iterable> current, - Collector> out) { - Iterator> iterator = current.iterator(); - if (!iterator.hasNext()) { - throw new RuntimeException("Error: Id not encountered before."); - } - - Tuple2 old = iterator.next(); - - long minimumComponentID = Long.MAX_VALUE; - - for (Tuple2 candidate : candidates) { - long candidateComponentID = candidate.f1; - if (candidateComponentID < minimumComponentID) { - minimumComponentID = candidateComponentID; - } - } - - if (minimumComponentID < old.f1) { - old.f1 = minimumComponentID; - out.collect(old); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java deleted file mode 100644 index 674a8fb37b5c4..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.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.test.iterative; - -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.aggregation.Aggregations; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.io.BufferedReader; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** Delta iteration test implementing the connected components algorithm with a join. */ -public class ConnectedComponentsITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - protected String verticesPath; - protected String edgesPath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - verticesPath = - createTempFile( - "vertices.txt", - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void testProgram() throws Exception { - // set up execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // read vertex and edge data - DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); - - DataSet> edges = - env.readCsvFile(edgesPath) - .fieldDelimiter(" ") - .types(Long.class, Long.class) - .flatMap(new ConnectedComponents.UndirectEdge()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); - - // apply the step logic: join with the edges, select the minimum neighbor, update if the - // component of the candidate is smaller - DataSet> changes = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new ConnectedComponents.NeighborWithComponentIDJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new ConnectedComponents.ComponentIdFilter()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(changes, changes); - - result.writeAsCsv(resultPath, "\n", " "); - - // execute program - env.execute("Connected Components Example"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - ConnectedComponentsData.checkOddEvenResult(reader); - } - } - - /** - * Duplicate the vertex ID into both fields of a {@link Tuple2}. - * - * @param key type - */ - public static final class DuplicateValue implements MapFunction, Tuple2> { - - @Override - public Tuple2 map(Tuple1 vertex) { - return new Tuple2<>(vertex.f0, vertex.f0); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java deleted file mode 100644 index 80b6de81e9845..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java +++ /dev/null @@ -1,175 +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.iterative; - -import org.apache.flink.api.common.functions.FlatJoinFunction; -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.aggregation.Aggregations; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.BufferedReader; -import java.util.Arrays; -import java.util.Collection; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** - * Delta iteration test implementing the connected components algorithm with a cogroup and join on - * the solution set. - */ -@RunWith(Parameterized.class) -public class ConnectedComponentsWithDeferredUpdateITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - private final boolean extraMapper; - - protected String verticesPath; - protected String edgesPath; - protected String resultPath; - - public ConnectedComponentsWithDeferredUpdateITCase(boolean extraMapper) { - this.extraMapper = extraMapper; - } - - @Override - protected void preSubmit() throws Exception { - verticesPath = - createTempFile( - "vertices.txt", - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void testProgram() throws Exception { - // set up execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // read vertex and edge data - DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); - - DataSet> edges = - env.readCsvFile(edgesPath) - .fieldDelimiter(" ") - .types(Long.class, Long.class) - .flatMap(new ConnectedComponents.UndirectEdge()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new ConnectedComponentsITCase.DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); - - // apply the step logic: join with the edges, select the minimum neighbor, update if the - // component of the candidate is smaller - DataSet> changes = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new ConnectedComponents.NeighborWithComponentIDJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new UpdateComponentIdMatchNonPreserving()); - - DataSet> delta; - if (extraMapper) { - delta = - changes.map( - // ID Mapper - new MapFunction, Tuple2>() { - private static final long serialVersionUID = -3929364091829757322L; - - @Override - public Tuple2 map(Tuple2 v) - throws Exception { - return v; - } - }); - } else { - delta = changes; - } - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(delta, changes); - - result.writeAsCsv(resultPath, "\n", " "); - - // execute program - env.execute("Connected Components Example"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - ConnectedComponentsData.checkOddEvenResult(reader); - } - } - - @Parameters - public static Collection getConfigurations() { - return Arrays.asList(new Object[] {false}, new Object[] {true}); - } - - private static final class UpdateComponentIdMatchNonPreserving - implements FlatJoinFunction< - Tuple2, Tuple2, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public void join( - Tuple2 candidate, - Tuple2 current, - Collector> out) - throws Exception { - - if (candidate.f1 < current.f1) { - out.collect(candidate); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java deleted file mode 100644 index 051bf7bc81e8b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java +++ /dev/null @@ -1,119 +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.iterative; - -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.graph.ConnectedComponents.ComponentIdFilter; -import org.apache.flink.examples.java.graph.ConnectedComponents.NeighborWithComponentIDJoin; -import org.apache.flink.examples.java.graph.ConnectedComponents.UndirectEdge; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.io.BufferedReader; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** Delta iteration test implementing the connected components algorithm with an object map. */ -@SuppressWarnings("serial") -public class ConnectedComponentsWithObjectMapITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - protected String verticesPath; - protected String edgesPath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - verticesPath = - createTempFile( - "vertices.txt", - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - ConnectedComponentsData.checkOddEvenResult(reader); - } - } - - @Override - protected void testProgram() throws Exception { - // set up execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // read vertex and edge data - DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); - - DataSet> edges = - env.readCsvFile(edgesPath) - .fieldDelimiter(" ") - .types(Long.class, Long.class) - .flatMap(new UndirectEdge()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new ConnectedComponentsITCase.DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); - iteration.setSolutionSetUnManaged(true); - - // apply the step logic: join with the edges, select the minimum neighbor, update if the - // component of the candidate is smaller - DataSet> changes = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new ComponentIdFilter()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(changes, changes); - - result.writeAsCsv(resultPath, "\n", " "); - - // execute program - env.execute("Connected Components Example"); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java deleted file mode 100644 index 11aa53447c109..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java +++ /dev/null @@ -1,148 +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.iterative; - -import org.apache.flink.api.common.functions.FlatJoinFunction; -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.functions.FunctionAnnotation; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.test.testdata.ConnectedComponentsData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import java.io.BufferedReader; - -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; - -/** - * Tests a bug that prevented that the solution set can be on both sides of the match/cogroup - * function. - */ -public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTestBaseJUnit4 { - - private static final long SEED = 0xBADC0FFEEBEEFL; - - private static final int NUM_VERTICES = 1000; - - private static final int NUM_EDGES = 10000; - - protected String verticesPath; - protected String edgesPath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - verticesPath = - createTempFile( - "vertices.txt", - ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES)); - edgesPath = - createTempFile( - "edges.txt", - ConnectedComponentsData.getRandomOddEvenEdges( - NUM_EDGES, NUM_VERTICES, SEED)); - resultPath = getTempFilePath("results"); - } - - @Override - protected void testProgram() throws Exception { - // set up execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // read vertex and edge data - DataSet> vertices = env.readCsvFile(verticesPath).types(Long.class); - - DataSet> edges = - env.readCsvFile(edgesPath) - .fieldDelimiter(" ") - .types(Long.class, Long.class) - .flatMap(new ConnectedComponents.UndirectEdge()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new ConnectedComponentsITCase.DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); - - // apply the step logic: join with the edges, select the minimum neighbor, update if the - // component of the candidate is smaller - DataSet> minNeighbor = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new ConnectedComponents.NeighborWithComponentIDJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1); - - DataSet> updatedIds = - iteration - .getSolutionSet() - .join(minNeighbor) - .where(0) - .equalTo(0) - .with(new UpdateComponentIdMatchMirrored()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(updatedIds, updatedIds); - - result.writeAsCsv(resultPath, "\n", " "); - - // execute program - env.execute("Connected Components Example"); - } - - @Override - protected void postSubmit() throws Exception { - for (BufferedReader reader : getResultReader(resultPath)) { - ConnectedComponentsData.checkOddEvenResult(reader); - } - } - - // -------------------------------------------------------------------------------------------- - // Classes and methods for the test program - // -------------------------------------------------------------------------------------------- - - @FunctionAnnotation.ForwardedFieldsSecond("*") - private static final class UpdateComponentIdMatchMirrored - implements FlatJoinFunction< - Tuple2, Tuple2, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public void join( - Tuple2 current, - Tuple2 candidate, - Collector> out) - throws Exception { - - if (candidate.f1 < current.f1) { - out.collect(candidate); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java deleted file mode 100644 index 3c3ce47083351..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java +++ /dev/null @@ -1,462 +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.iterative; - -import org.apache.flink.api.common.aggregators.Aggregator; -import org.apache.flink.api.common.aggregators.ConvergenceCriterion; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichCoGroupFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.types.Value; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.IOException; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** Implementation of PageRank accounting for "sink" vertices with 0 out-degree. */ -@RunWith(Parameterized.class) -@SuppressWarnings({"serial", "unchecked"}) -public class DanglingPageRankITCase extends MultipleProgramsTestBaseJUnit4 { - - private static final String AGGREGATOR_NAME = "pagerank.aggregator"; - - public DanglingPageRankITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testDanglingPageRank() { - try { - final int numIterations = 25; - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> vertices = - env.fromElements( - new Tuple2<>(1L, false), - new Tuple2<>(2L, false), - new Tuple2<>(5L, false), - new Tuple2<>(3L, true), - new Tuple2<>(4L, false)); - - DataSet edges = - env.fromElements( - new PageWithLinks(2L, new long[] {1}), - new PageWithLinks(5L, new long[] {2, 4}), - new PageWithLinks(4L, new long[] {3, 2}), - new PageWithLinks(1L, new long[] {4, 2, 3})); - - final long numVertices = vertices.count(); - final long numDanglingVertices = - vertices.filter( - new FilterFunction>() { - @Override - public boolean filter(Tuple2 value) { - return value.f1; - } - }) - .count(); - - DataSet verticesWithInitialRank = - vertices.map( - new MapFunction, PageWithRankAndDangling>() { - - @Override - public PageWithRankAndDangling map(Tuple2 value) { - return new PageWithRankAndDangling( - value.f0, 1.0 / numVertices, value.f1); - } - }); - - IterativeDataSet iteration = - verticesWithInitialRank.iterate(numIterations); - - iteration - .getAggregators() - .registerAggregationConvergenceCriterion( - AGGREGATOR_NAME, - new PageRankStatsAggregator(), - new DiffL1NormConvergenceCriterion()); - - DataSet partialRanks = - iteration - .join(edges) - .where("pageId") - .equalTo("pageId") - .with( - new FlatJoinFunction< - PageWithRankAndDangling, - PageWithLinks, - PageWithRank>() { - - @Override - public void join( - PageWithRankAndDangling page, - PageWithLinks links, - Collector out) { - - double rankToDistribute = - page.rank / (double) links.targets.length; - PageWithRank output = - new PageWithRank(0L, rankToDistribute); - - for (long target : links.targets) { - output.pageId = target; - out.collect(output); - } - } - }); - - DataSet newRanks = - iteration - .coGroup(partialRanks) - .where("pageId") - .equalTo("pageId") - .with( - new RichCoGroupFunction< - PageWithRankAndDangling, - PageWithRank, - PageWithRankAndDangling>() { - - private static final double BETA = 0.85; - - private final double randomJump = - (1.0 - BETA) / numVertices; - private PageRankStatsAggregator aggregator; - private double danglingRankFactor; - - @Override - public void open(OpenContext openContext) throws Exception { - int currentIteration = - getIterationRuntimeContext() - .getSuperstepNumber(); - - aggregator = - getIterationRuntimeContext() - .getIterationAggregator( - AGGREGATOR_NAME); - - if (currentIteration == 1) { - danglingRankFactor = - BETA - * (double) numDanglingVertices - / ((double) numVertices - * (double) numVertices); - } else { - PageRankStats previousAggregate = - getIterationRuntimeContext() - .getPreviousIterationAggregate( - AGGREGATOR_NAME); - danglingRankFactor = - BETA - * previousAggregate.danglingRank() - / (double) numVertices; - } - } - - @Override - public void coGroup( - Iterable currentPages, - Iterable partialRanks, - Collector out) { - - // compute the next rank - long edges = 0; - double summedRank = 0; - for (PageWithRank partial : partialRanks) { - summedRank += partial.rank; - edges++; - } - double rank = - BETA * summedRank - + randomJump - + danglingRankFactor; - - // current rank, for stats and convergence - PageWithRankAndDangling currentPage = - currentPages.iterator().next(); - double currentRank = currentPage.rank; - boolean isDangling = currentPage.dangling; - - // maintain statistics to compensate for probability - // loss on dangling nodes - double danglingRankToAggregate = isDangling ? rank : 0; - long danglingVerticesToAggregate = isDangling ? 1 : 0; - double diff = Math.abs(currentRank - rank); - aggregator.aggregate( - diff, - rank, - danglingRankToAggregate, - danglingVerticesToAggregate, - 1, - edges); - - currentPage.rank = rank; - out.collect(currentPage); - } - }); - - List result = iteration.closeWith(newRanks).collect(); - - double totalRank = 0.0; - for (PageWithRankAndDangling r : result) { - totalRank += r.rank; - assertTrue(r.pageId >= 1 && r.pageId <= 5); - assertTrue(r.pageId != 3 || r.dangling); - } - - assertEquals(1.0, totalRank, 0.001); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // ------------------------------------------------------------------------ - // custom types - // ------------------------------------------------------------------------ - - /** POJO for page ID and rank value. */ - public static class PageWithRank { - - public long pageId; - public double rank; - - public PageWithRank() {} - - public PageWithRank(long pageId, double rank) { - this.pageId = pageId; - this.rank = rank; - } - } - - /** POJO for page ID, rank value, and whether a "dangling" vertex with 0 out-degree. */ - public static class PageWithRankAndDangling { - - public long pageId; - public double rank; - public boolean dangling; - - public PageWithRankAndDangling() {} - - public PageWithRankAndDangling(long pageId, double rank, boolean dangling) { - this.pageId = pageId; - this.rank = rank; - this.dangling = dangling; - } - - @Override - public String toString() { - return "PageWithRankAndDangling{" - + "pageId=" - + pageId - + ", rank=" - + rank - + ", dangling=" - + dangling - + '}'; - } - } - - /** POJO for page ID and list of target IDs. */ - public static class PageWithLinks { - - public long pageId; - public long[] targets; - - public PageWithLinks() {} - - public PageWithLinks(long pageId, long[] targets) { - this.pageId = pageId; - this.targets = targets; - } - } - - // ------------------------------------------------------------------------ - // statistics - // ------------------------------------------------------------------------ - - /** PageRank statistics. */ - public static class PageRankStats implements Value { - - private double diff; - private double rank; - private double danglingRank; - private long numDanglingVertices; - private long numVertices; - private long edges; - - public PageRankStats() {} - - public PageRankStats( - double diff, - double rank, - double danglingRank, - long numDanglingVertices, - long numVertices, - long edges) { - - this.diff = diff; - this.rank = rank; - this.danglingRank = danglingRank; - this.numDanglingVertices = numDanglingVertices; - this.numVertices = numVertices; - this.edges = edges; - } - - public double diff() { - return diff; - } - - public double rank() { - return rank; - } - - public double danglingRank() { - return danglingRank; - } - - public long numDanglingVertices() { - return numDanglingVertices; - } - - public long numVertices() { - return numVertices; - } - - public long edges() { - return edges; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.writeDouble(diff); - out.writeDouble(rank); - out.writeDouble(danglingRank); - out.writeLong(numDanglingVertices); - out.writeLong(numVertices); - out.writeLong(edges); - } - - @Override - public void read(DataInputView in) throws IOException { - diff = in.readDouble(); - rank = in.readDouble(); - danglingRank = in.readDouble(); - numDanglingVertices = in.readLong(); - numVertices = in.readLong(); - edges = in.readLong(); - } - - @Override - public String toString() { - return "PageRankStats: diff [" - + diff - + "], rank [" - + rank - + "], danglingRank [" - + danglingRank - + "], numDanglingVertices [" - + numDanglingVertices - + "], numVertices [" - + numVertices - + "], edges [" - + edges - + "]"; - } - } - - private static class PageRankStatsAggregator implements Aggregator { - - private double diff; - private double rank; - private double danglingRank; - private long numDanglingVertices; - private long numVertices; - private long edges; - - @Override - public PageRankStats getAggregate() { - return new PageRankStats( - diff, rank, danglingRank, numDanglingVertices, numVertices, edges); - } - - public void aggregate( - double diffDelta, - double rankDelta, - double danglingRankDelta, - long danglingVerticesDelta, - long verticesDelta, - long edgesDelta) { - diff += diffDelta; - rank += rankDelta; - danglingRank += danglingRankDelta; - numDanglingVertices += danglingVerticesDelta; - numVertices += verticesDelta; - edges += edgesDelta; - } - - @Override - public void aggregate(PageRankStats pageRankStats) { - diff += pageRankStats.diff(); - rank += pageRankStats.rank(); - danglingRank += pageRankStats.danglingRank(); - numDanglingVertices += pageRankStats.numDanglingVertices(); - numVertices += pageRankStats.numVertices(); - edges += pageRankStats.edges(); - } - - @Override - public void reset() { - diff = 0; - rank = 0; - danglingRank = 0; - numDanglingVertices = 0; - numVertices = 0; - edges = 0; - } - } - - private static class DiffL1NormConvergenceCriterion - implements ConvergenceCriterion { - - private static final double EPSILON = 0.00005; - - @Override - public boolean isConverged(int iteration, PageRankStats pageRankStats) { - return pageRankStats.diff() < EPSILON; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java deleted file mode 100644 index 7ffc8cee6d6f2..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java +++ /dev/null @@ -1,90 +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.iterative; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** Test delta iterations that do not join with the solution set. */ -@SuppressWarnings("serial") -public class DeltaIterationNotDependingOnSolutionSetITCase extends JavaProgramTestBaseJUnit4 { - private final List> result = new ArrayList<>(); - - @Override - protected void testProgram() throws Exception { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> input = - env.generateSequence(0, 9).map(new Duplicator()); - - DeltaIteration, Tuple2> iteration = - input.iterateDelta(input, 5, 1); - - iteration - .closeWith(iteration.getWorkset(), iteration.getWorkset().map(new TestMapper())) - .output(new LocalCollectionOutputFormat>(result)); - - env.execute(); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Override - protected void postSubmit() { - boolean[] present = new boolean[50]; - for (Tuple2 t : result) { - present[t.f0.intValue()] = true; - } - - for (int i = 0; i < present.length; i++) { - assertTrue(String.format("Missing tuple (%d, %d)", i, i), present[i]); - } - } - - private static final class Duplicator implements MapFunction> { - @Override - public Tuple2 map(T value) { - return new Tuple2(value, value); - } - } - - private static final class TestMapper - extends RichMapFunction, Tuple2> { - @Override - public Tuple2 map(Tuple2 value) { - return new Tuple2<>(value.f0 + 10, value.f1 + 10); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java deleted file mode 100644 index 3838c42198f00..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java +++ /dev/null @@ -1,265 +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.iterative; - -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.common.functions.RichJoinFunction; -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.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** - * Iterative Connected Components test case which recomputes only the elements of the solution set - * whose at least one dependency (in-neighbor) has changed since the last iteration. Requires two - * joins with the solution set. - */ -@SuppressWarnings("serial") -public class DependencyConnectedComponentsITCase extends JavaProgramTestBaseJUnit4 { - - private static final int MAX_ITERATIONS = 20; - private static final int parallelism = 1; - - protected static List> verticesInput = new ArrayList>(); - protected static List> edgesInput = new ArrayList>(); - private String resultPath; - private String expectedResult; - - @Override - protected void preSubmit() throws Exception { - verticesInput.clear(); - edgesInput.clear(); - - // vertices input - verticesInput.add(new Tuple2<>(1L, 1L)); - verticesInput.add(new Tuple2<>(2L, 2L)); - verticesInput.add(new Tuple2<>(3L, 3L)); - verticesInput.add(new Tuple2<>(4L, 4L)); - verticesInput.add(new Tuple2<>(5L, 5L)); - verticesInput.add(new Tuple2<>(6L, 6L)); - verticesInput.add(new Tuple2<>(7L, 7L)); - verticesInput.add(new Tuple2<>(8L, 8L)); - verticesInput.add(new Tuple2<>(9L, 9L)); - - // vertices input - edgesInput.add(new Tuple2<>(1L, 2L)); - edgesInput.add(new Tuple2<>(1L, 3L)); - edgesInput.add(new Tuple2<>(2L, 3L)); - edgesInput.add(new Tuple2<>(2L, 4L)); - edgesInput.add(new Tuple2<>(2L, 1L)); - edgesInput.add(new Tuple2<>(3L, 1L)); - edgesInput.add(new Tuple2<>(3L, 2L)); - edgesInput.add(new Tuple2<>(4L, 2L)); - edgesInput.add(new Tuple2<>(4L, 6L)); - edgesInput.add(new Tuple2<>(5L, 6L)); - edgesInput.add(new Tuple2<>(6L, 4L)); - edgesInput.add(new Tuple2<>(6L, 5L)); - edgesInput.add(new Tuple2<>(7L, 8L)); - edgesInput.add(new Tuple2<>(7L, 9L)); - edgesInput.add(new Tuple2<>(8L, 7L)); - edgesInput.add(new Tuple2<>(8L, 9L)); - edgesInput.add(new Tuple2<>(9L, 7L)); - edgesInput.add(new Tuple2<>(9L, 8L)); - - resultPath = getTempDirPath("result"); - - expectedResult = - "(1,1)\n" + "(2,1)\n" + "(3,1)\n" + "(4,1)\n" + "(5,1)\n" + "(6,1)\n" + "(7,7)\n" - + "(8,7)\n" + "(9,7)\n"; - } - - @Override - protected void testProgram() throws Exception { - DependencyConnectedComponentsProgram.runProgram(resultPath); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expectedResult, resultPath); - } - - private static class DependencyConnectedComponentsProgram { - - public static String runProgram(String resultPath) throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> initialSolutionSet = env.fromCollection(verticesInput); - DataSet> edges = env.fromCollection(edgesInput); - int keyPosition = 0; - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta( - initialSolutionSet, MAX_ITERATIONS, keyPosition); - - DataSet candidates = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new FindCandidatesJoin()) - .groupBy( - new KeySelector() { - public Long getKey(Long id) { - return id; - } - }) - .reduceGroup(new RemoveDuplicatesReduce()); - - DataSet> candidatesDependencies = - candidates - .join(edges) - .where( - new KeySelector() { - public Long getKey(Long id) { - return id; - } - }) - .equalTo( - new KeySelector, Long>() { - public Long getKey(Tuple2 vertexWithId) { - return vertexWithId.f1; - } - }) - .with(new FindCandidatesDependenciesJoin()); - - DataSet> verticesWithNewComponents = - candidatesDependencies - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .groupBy(0) - .reduceGroup(new MinimumReduce()); - - DataSet> updatedComponentId = - verticesWithNewComponents - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .flatMap(new MinimumIdFilter()); - - iteration.closeWith(updatedComponentId, updatedComponentId).writeAsText(resultPath); - - env.execute(); - - return resultPath; - } - } - - private static final class FindCandidatesJoin - extends RichJoinFunction, Tuple2, Long> { - - private static final long serialVersionUID = 1L; - - @Override - public Long join(Tuple2 vertexWithCompId, Tuple2 edge) - throws Exception { - // emit target vertex - return edge.f1; - } - } - - private static final class RemoveDuplicatesReduce extends RichGroupReduceFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - out.collect(values.iterator().next()); - } - } - - private static final class FindCandidatesDependenciesJoin - extends RichJoinFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 join(Long candidateId, Tuple2 edge) throws Exception { - return edge; - } - } - - private static final class NeighborWithComponentIDJoin - extends RichJoinFunction, Tuple2, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 join(Tuple2 edge, Tuple2 vertexWithCompId) - throws Exception { - - vertexWithCompId.setField(edge.f1, 0); - return vertexWithCompId; - } - } - - private static final class MinimumReduce - extends RichGroupReduceFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - final Tuple2 resultVertex = new Tuple2(); - - @Override - public void reduce(Iterable> values, Collector> out) { - Long vertexId = 0L; - Long minimumCompId = Long.MAX_VALUE; - - for (Tuple2 value : values) { - vertexId = value.f0; - Long candidateCompId = value.f1; - if (candidateCompId < minimumCompId) { - minimumCompId = candidateCompId; - } - } - resultVertex.f0 = vertexId; - resultVertex.f1 = minimumCompId; - - out.collect(resultVertex); - } - } - - private static final class MinimumIdFilter - extends RichFlatMapFunction< - Tuple2, Tuple2>, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public void flatMap( - Tuple2, Tuple2> vertexWithNewAndOldId, - Collector> out) { - if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) { - out.collect(vertexWithNewAndOldId.f0); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java deleted file mode 100644 index 859ef8ae33d81..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.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.test.iterative; - -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.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.ArrayList; -import java.util.List; - -/** Test empty (identity) delta iteration. */ -@SuppressWarnings("serial") -public class EmptyWorksetIterationITCase extends JavaProgramTestBaseJUnit4 { - - private List> result = new ArrayList>(); - - @Override - protected void testProgram() throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = env.generateSequence(1, 20).map(new Dupl()); - - DeltaIteration, Tuple2> iter = - input.iterateDelta(input, 20, 0); - iter.closeWith(iter.getWorkset(), iter.getWorkset()) - .output(new LocalCollectionOutputFormat>(result)); - - env.execute(); - } - - private static final class Dupl implements MapFunction> { - - @Override - public Tuple2 map(Long value) { - return new Tuple2(value, value); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java deleted file mode 100644 index fce2eae7b8862..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java +++ /dev/null @@ -1,56 +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.iterative; - -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.ArrayList; -import java.util.List; - -import static org.junit.Assert.assertEquals; - -/** Test empty (identity) bulk iteration. */ -public class IdentityIterationITCase extends JavaProgramTestBaseJUnit4 { - - private List result = new ArrayList(); - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - IterativeDataSet iteration = env.generateSequence(1, 10).iterate(100); - iteration.closeWith(iteration).output(new LocalCollectionOutputFormat(result)); - - env.execute(); - } - - @Override - protected void postSubmit() { - assertEquals(10, result.size()); - - long sum = 0; - for (Long l : result) { - sum += l; - } - assertEquals(55, sum); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java deleted file mode 100644 index 676304f2f2bbc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java +++ /dev/null @@ -1,102 +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.iterative; - -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -/** - * Test where the test data is constructed such that the merge join zig zag has an early out, - * leaving elements on the dynamic path input unconsumed. - */ -@SuppressWarnings("serial") -public class IterationIncompleteDynamicPathConsumptionITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // the test data is constructed such that the merge join zig zag - // has an early out, leaving elements on the dynamic path input unconsumed - - DataSet edges = - env.fromElements( - new Path(1, 2), - new Path(1, 4), - new Path(3, 6), - new Path(3, 8), - new Path(1, 10), - new Path(1, 12), - new Path(3, 14), - new Path(3, 16), - new Path(1, 18), - new Path(1, 20)); - - IterativeDataSet currentPaths = edges.iterate(10); - - DataSet newPaths = - currentPaths - .join(edges, JoinHint.REPARTITION_SORT_MERGE) - .where("to") - .equalTo("from") - .with(new PathConnector()) - .union(currentPaths) - .distinct("from", "to"); - - DataSet result = currentPaths.closeWith(newPaths); - - result.output(new DiscardingOutputFormat()); - - env.execute(); - } - - private static class PathConnector implements JoinFunction { - - @Override - public Path join(Path path, Path edge) { - return new Path(path.from, edge.to); - } - } - - // -------------------------------------------------------------------------------------------- - - /** Simple POJO. */ - public static class Path { - - public long from; - public long to; - - public Path() {} - - public Path(long from, long to) { - this.from = from; - this.to = to; - } - - @Override - public String toString() { - return "(" + from + "," + to + ")"; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java deleted file mode 100644 index af701a6ebac9a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java +++ /dev/null @@ -1,102 +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.iterative; - -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -/** - * Test where the test data is constructed such that the merge join zig zag has an early out, - * leaving elements on the static path input unconsumed. - */ -@SuppressWarnings("serial") -public class IterationIncompleteStaticPathConsumptionITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // the test data is constructed such that the merge join zig zag - // has an early out, leaving elements on the static path input unconsumed - - DataSet edges = - env.fromElements( - new Path(2, 1), - new Path(4, 1), - new Path(6, 3), - new Path(8, 3), - new Path(10, 1), - new Path(12, 1), - new Path(14, 3), - new Path(16, 3), - new Path(18, 1), - new Path(20, 1)); - - IterativeDataSet currentPaths = edges.iterate(10); - - DataSet newPaths = - currentPaths - .join(edges, JoinHint.REPARTITION_SORT_MERGE) - .where("to") - .equalTo("from") - .with(new PathConnector()) - .union(currentPaths) - .distinct("from", "to"); - - DataSet result = currentPaths.closeWith(newPaths); - - result.output(new DiscardingOutputFormat()); - - env.execute(); - } - - private static class PathConnector implements JoinFunction { - - @Override - public Path join(Path path, Path edge) { - return new Path(path.from, edge.to); - } - } - - // -------------------------------------------------------------------------------------------- - - /** Simple POJO. */ - public static class Path { - - public long from; - public long to; - - public Path() {} - - public Path(long from, long to) { - this.from = from; - this.to = to; - } - - @Override - public String toString() { - return "(" + from + "," + to + ")"; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java deleted file mode 100644 index dc2ec1f68f9cc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java +++ /dev/null @@ -1,80 +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.iterative; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.containsResultAsText; - -/** Test iteration with termination criterion. */ -public class IterationTerminationWithTerminationTail extends JavaProgramTestBaseJUnit4 { - private static final String EXPECTED = "22\n"; - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet initialInput = env.fromElements("1", "2", "3", "4", "5").name("input"); - - IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); - - DataSet sumReduce = - iteration.reduceGroup(new SumReducer()).name("Compute sum (GroupReduce"); - - DataSet terminationFilter = - sumReduce - .filter(new TerminationFilter()) - .name("Compute termination criterion (Map)"); - - List result = iteration.closeWith(sumReduce, terminationFilter).collect(); - - containsResultAsText(result, EXPECTED); - } - - private static final class SumReducer implements GroupReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) throws Exception { - int sum = 0; - for (String value : values) { - sum += Integer.parseInt(value) + 1; - } - out.collect("" + sum); - } - } - - private static class TerminationFilter implements FilterFunction { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(String value) throws Exception { - return Integer.parseInt(value) < 22; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java deleted file mode 100644 index 871352aad21a0..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java +++ /dev/null @@ -1,80 +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.iterative; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.containsResultAsText; - -/** Test iteration with termination criterion consuming the iteration tail. */ -public class IterationTerminationWithTwoTails extends JavaProgramTestBaseJUnit4 { - private static final String EXPECTED = "22\n"; - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet initialInput = env.fromElements("1", "2", "3", "4", "5").name("input"); - - IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); - - DataSet sumReduce = - iteration.reduceGroup(new SumReducer()).name("Compute sum (GroupReduce"); - - DataSet terminationFilter = - iteration - .filter(new TerminationFilter()) - .name("Compute termination criterion (Map)"); - - List result = iteration.closeWith(sumReduce, terminationFilter).collect(); - - containsResultAsText(result, EXPECTED); - } - - private static final class SumReducer implements GroupReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) throws Exception { - int sum = 0; - for (String value : values) { - sum += Integer.parseInt(value) + 1; - } - out.collect("" + sum); - } - } - - private static class TerminationFilter implements FilterFunction { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(String value) throws Exception { - return Integer.parseInt(value) < 21; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java deleted file mode 100644 index 94df3e4ca7e23..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.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.test.iterative; - -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; - -/** Test iterator with an all-reduce. */ -public class IterationWithAllReducerITCase extends JavaProgramTestBaseJUnit4 { - private static final String EXPECTED = "1\n"; - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet initialInput = env.fromElements("1", "1", "1", "1", "1", "1", "1", "1"); - - IterativeDataSet iteration = initialInput.iterate(5).name("Loop"); - - DataSet sumReduce = - iteration - .reduce( - new ReduceFunction() { - @Override - public String reduce(String value1, String value2) - throws Exception { - return value1; - } - }) - .name("Compute sum (Reduce)"); - - List result = iteration.closeWith(sumReduce).collect(); - - compareResultAsText(result, EXPECTED); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java deleted file mode 100644 index 4d92d2d86a699..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java +++ /dev/null @@ -1,95 +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.iterative; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -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.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.CoordVector; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.test.util.PointFormatter; -import org.apache.flink.test.util.PointInFormat; -import org.apache.flink.util.Collector; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Test iteration with operator chaining. */ -public class IterationWithChainingITCase extends JavaProgramTestBaseJUnit4 { - - private static final String DATA_POINTS = - "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n"; - - private String dataPath; - private String resultPath; - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("data_points.txt", DATA_POINTS); - resultPath = getTempFilePath("result"); - } - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet> initialInput = - env.readFile(new PointInFormat(), dataPath).setParallelism(1).name("Input"); - - IterativeDataSet> iteration = - initialInput.iterate(2).name("Loop"); - - DataSet> identity = - iteration - .groupBy(0) - .reduceGroup( - new GroupReduceFunction< - Tuple2, - Tuple2>() { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - for (Tuple2 value : values) { - out.collect(value); - } - } - }) - .map( - new MapFunction< - Tuple2, - Tuple2>() { - @Override - public Tuple2 map( - Tuple2 value) throws Exception { - return value; - } - }); - - iteration.closeWith(identity).writeAsFormattedText(resultPath, new PointFormatter()); - - env.execute("Iteration with chained map test"); - - compareResultsByLinesInMemory(DATA_POINTS, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java deleted file mode 100644 index a3b8da2d5ffc8..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java +++ /dev/null @@ -1,101 +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.iterative; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -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.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.CoordVector; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.test.util.PointFormatter; -import org.apache.flink.test.util.PointInFormat; -import org.apache.flink.util.Collector; - -import java.io.Serializable; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Test iteration with union. */ -public class IterationWithUnionITCase extends JavaProgramTestBaseJUnit4 { - - private static final String DATAPOINTS = - "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n"; - - protected String dataPath; - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - dataPath = createTempFile("datapoints.txt", DATAPOINTS); - resultPath = getTempDirPath("union_iter_result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory( - DATAPOINTS + DATAPOINTS + DATAPOINTS + DATAPOINTS, resultPath); - } - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> initialInput = - env.readFile(new PointInFormat(), this.dataPath).setParallelism(1); - - IterativeDataSet> iteration = initialInput.iterate(2); - - DataSet> result = - iteration.union(iteration).map(new IdentityMapper()); - - iteration.closeWith(result).writeAsFormattedText(this.resultPath, new PointFormatter()); - - env.execute(); - } - - static final class IdentityMapper - implements MapFunction, Tuple2>, - Serializable { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 map(Tuple2 rec) { - return rec; - } - } - - static class DummyReducer - implements GroupReduceFunction< - Tuple2, Tuple2>, - Serializable { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> it, - Collector> out) { - for (Tuple2 r : it) { - out.collect(r); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java deleted file mode 100644 index 2cec06b24aa67..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.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.test.iterative; - -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.operators.IterativeDataSet; -import org.apache.flink.examples.java.clustering.KMeans; -import org.apache.flink.examples.java.clustering.KMeans.Centroid; -import org.apache.flink.examples.java.clustering.KMeans.Point; -import org.apache.flink.test.testdata.KMeansData; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.List; -import java.util.Locale; - -/** Test KMeans clustering with a broadcast set. */ -public class KMeansWithBroadcastSetITCase extends JavaProgramTestBaseJUnit4 { - - @SuppressWarnings("serial") - @Override - protected void testProgram() throws Exception { - - String[] points = KMeansData.DATAPOINTS_2D.split("\n"); - String[] centers = KMeansData.INITIAL_CENTERS_2D.split("\n"); - - // set up execution environment - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet pointsSet = - env.fromElements(points) - .map( - new MapFunction() { - public Point map(String p) { - String[] fields = p.split("\\|"); - return new Point( - Double.parseDouble(fields[1]), - Double.parseDouble(fields[2])); - } - }); - - DataSet centroidsSet = - env.fromElements(centers) - .map( - new MapFunction() { - public Centroid map(String c) { - String[] fields = c.split("\\|"); - return new Centroid( - Integer.parseInt(fields[0]), - Double.parseDouble(fields[1]), - Double.parseDouble(fields[2])); - } - }); - - // set number of bulk iterations for KMeans algorithm - IterativeDataSet loop = centroidsSet.iterate(20); - - DataSet newCentroids = - pointsSet - // compute closest centroid for each point - .map(new KMeans.SelectNearestCenter()) - .withBroadcastSet(loop, "centroids") - // count and sum point coordinates for each centroid - .map(new KMeans.CountAppender()) - .groupBy(0) - .reduce(new KMeans.CentroidAccumulator()) - // compute new centroids from point counts and coordinate sums - .map(new KMeans.CentroidAverager()); - - // feed new centroids back into next iteration - DataSet finalCentroids = loop.closeWith(newCentroids); - - DataSet stringCentroids = - finalCentroids.map( - new MapFunction() { - @Override - public String map(Centroid c) throws Exception { - return String.format(Locale.US, "%d|%.2f|%.2f|", c.id, c.x, c.y); - } - }); - - List result = stringCentroids.collect(); - - KMeansData.checkResultsWithDelta( - KMeansData.CENTERS_2D_AFTER_20_ITERATIONS_DOUBLE_DIGIT, result, 0.01); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java deleted file mode 100644 index f63db756b7eff..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java +++ /dev/null @@ -1,67 +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.iterative; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.optimizer.iterations.MultipleJoinsWithSolutionSetCompilerTest; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import org.junit.Assert; - -import java.util.ArrayList; -import java.util.List; - -/** Test multiple joins with the solution set. */ -public class MultipleSolutionSetJoinsITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - - final int numIters = 4; - final double expectedFactor = (int) Math.pow(7, numIters); - - // this is an artificial program, it does not compute anything sensical - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> initialData = - env.fromElements( - new Tuple2(1L, 1.0), - new Tuple2(2L, 2.0), - new Tuple2(3L, 3.0), - new Tuple2(4L, 4.0), - new Tuple2(5L, 5.0), - new Tuple2(6L, 6.0)); - - DataSet> result = - MultipleJoinsWithSolutionSetCompilerTest.constructPlan(initialData, numIters); - - List> resultCollector = new ArrayList>(); - result.output(new LocalCollectionOutputFormat<>(resultCollector)); - - env.execute(); - - for (Tuple2 tuple : resultCollector) { - Assert.assertEquals(expectedFactor * tuple.f0, tuple.f1.doubleValue(), 0.0); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java deleted file mode 100644 index aa90366b9e381..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java +++ /dev/null @@ -1,92 +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.iterative; - -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.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** Test for duplicate elimination in the solution set. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class SolutionSetDuplicatesITCase extends MultipleProgramsTestBaseJUnit4 { - - public SolutionSetDuplicatesITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testProgram() { - try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = - env.generateSequence(0, 10) - .flatMap( - new FlatMapFunction>() { - @Override - public void flatMap( - Long value, Collector> out) { - out.collect(new Tuple2(value, value)); - out.collect(new Tuple2(value, value)); - out.collect(new Tuple2(value, value)); - } - }) - .rebalance(); - - DeltaIteration, Tuple2> iter = - data.iterateDelta(data, 10, 0); - - List result = - iter.closeWith(iter.getWorkset(), iter.getWorkset()) - .map( - new MapFunction, Integer>() { - @Override - public Integer map(Tuple2 value) { - return value.f0.intValue(); - } - }) - .collect(); - - assertEquals(11, result.size()); - - Collections.sort(result); - assertEquals(Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10), result); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java deleted file mode 100644 index 9be3cef0f9e96..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java +++ /dev/null @@ -1,89 +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.iterative; - -import org.apache.flink.api.common.functions.JoinFunction; -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.functions.KeySelector; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -/** Test iterations referenced from the static path of other iterations. */ -public class StaticlyNestedIterationsITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet data1 = env.generateSequence(1, 100); - DataSet data2 = env.generateSequence(1, 100); - - IterativeDataSet firstIteration = data1.iterate(100); - - DataSet firstResult = firstIteration.closeWith(firstIteration.map(new IdMapper())); - - IterativeDataSet mainIteration = data2.map(new IdMapper()).iterate(100); - - DataSet joined = - mainIteration - .join(firstResult) - .where(new IdKeyExtractor()) - .equalTo(new IdKeyExtractor()) - .with(new Joiner()); - - DataSet mainResult = mainIteration.closeWith(joined); - - mainResult.output(new DiscardingOutputFormat()); - - env.execute(); - } - - private static class IdKeyExtractor implements KeySelector { - - private static final long serialVersionUID = 1L; - - @Override - public Long getKey(Long value) { - return value; - } - } - - private static class IdMapper implements MapFunction { - - private static final long serialVersionUID = 1L; - - @Override - public Long map(Long value) { - return value; - } - } - - private static class Joiner implements JoinFunction { - - private static final long serialVersionUID = 1L; - - @Override - public Long join(Long first, Long second) { - return first; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java deleted file mode 100644 index 74173187d350b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java +++ /dev/null @@ -1,58 +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.iterative; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.ArrayList; - -import static org.junit.Assert.assertEquals; - -/** Test union between static and dynamic path in an iteration. */ -public class UnionStaticDynamicIterationITCase extends JavaProgramTestBaseJUnit4 { - - private final ArrayList result = new ArrayList(); - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet inputStatic = env.generateSequence(1, 4); - DataSet inputIteration = env.generateSequence(1, 4); - - IterativeDataSet iteration = inputIteration.iterate(3); - - DataSet result = - iteration.closeWith( - inputStatic.union(inputStatic).union(iteration.union(iteration))); - - result.output(new LocalCollectionOutputFormat(this.result)); - - env.execute(); - } - - @Override - protected void postSubmit() throws Exception { - assertEquals(88, result.size()); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java deleted file mode 100644 index 772b37688bfb7..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java +++ /dev/null @@ -1,377 +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.iterative.aggregators; - -import org.apache.flink.api.common.aggregators.ConvergenceCriterion; -import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.functions.RichJoinFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.LongValue; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.junit.Assert.assertEquals; - -/** Connected Components test case that uses a parameterizable convergence criterion. */ -@RunWith(Parameterized.class) -@SuppressWarnings("serial") -public class AggregatorConvergenceITCase extends MultipleProgramsTestBaseJUnit4 { - - public AggregatorConvergenceITCase(TestExecutionMode mode) { - super(mode); - } - - final List> verticesInput = - Arrays.asList( - new Tuple2<>(1L, 1L), - new Tuple2<>(2L, 2L), - new Tuple2<>(3L, 3L), - new Tuple2<>(4L, 4L), - new Tuple2<>(5L, 5L), - new Tuple2<>(6L, 6L), - new Tuple2<>(7L, 7L), - new Tuple2<>(8L, 8L), - new Tuple2<>(9L, 9L)); - - final List> edgesInput = - Arrays.asList( - new Tuple2<>(1L, 2L), - new Tuple2<>(1L, 3L), - new Tuple2<>(2L, 3L), - new Tuple2<>(2L, 4L), - new Tuple2<>(2L, 1L), - new Tuple2<>(3L, 1L), - new Tuple2<>(3L, 2L), - new Tuple2<>(4L, 2L), - new Tuple2<>(4L, 6L), - new Tuple2<>(5L, 6L), - new Tuple2<>(6L, 4L), - new Tuple2<>(6L, 5L), - new Tuple2<>(7L, 8L), - new Tuple2<>(7L, 9L), - new Tuple2<>(8L, 7L), - new Tuple2<>(8L, 9L), - new Tuple2<>(9L, 7L), - new Tuple2<>(9L, 8L)); - - final List> expectedResult = - Arrays.asList( - new Tuple2<>(1L, 1L), - new Tuple2<>(2L, 1L), - new Tuple2<>(3L, 1L), - new Tuple2<>(4L, 1L), - new Tuple2<>(5L, 2L), - new Tuple2<>(6L, 1L), - new Tuple2<>(7L, 7L), - new Tuple2<>(8L, 7L), - new Tuple2<>(9L, 7L)); - - @Test - public void testConnectedComponentsWithParametrizableConvergence() throws Exception { - - // name of the aggregator that checks for convergence - final String updatedElements = "updated.elements.aggr"; - - // the iteration stops if less than this number of elements change value - final long convergenceThreshold = 3; - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> initialSolutionSet = env.fromCollection(verticesInput); - DataSet> edges = env.fromCollection(edgesInput); - - IterativeDataSet> iteration = initialSolutionSet.iterate(10); - - // register the convergence criterion - iteration.registerAggregationConvergenceCriterion( - updatedElements, - new LongSumAggregator(), - new UpdatedElementsConvergenceCriterion(convergenceThreshold)); - - DataSet> verticesWithNewComponents = - iteration - .join(edges) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .groupBy(0) - .min(1); - - DataSet> updatedComponentId = - verticesWithNewComponents - .join(iteration) - .where(0) - .equalTo(0) - .flatMap(new MinimumIdFilter(updatedElements)); - - List> result = iteration.closeWith(updatedComponentId).collect(); - Collections.sort(result, new TestBaseUtils.TupleComparator>()); - - assertEquals(expectedResult, result); - } - - @Test - public void testDeltaConnectedComponentsWithParametrizableConvergence() throws Exception { - - // name of the aggregator that checks for convergence - final String updatedElements = "updated.elements.aggr"; - - // the iteration stops if less than this number of elements change value - final long convergenceThreshold = 3; - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> initialSolutionSet = env.fromCollection(verticesInput); - DataSet> edges = env.fromCollection(edgesInput); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialSolutionSet, 10, 0); - - // register the convergence criterion - iteration.registerAggregationConvergenceCriterion( - updatedElements, - new LongSumAggregator(), - new UpdatedElementsConvergenceCriterion(convergenceThreshold)); - - DataSet> verticesWithNewComponents = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .groupBy(0) - .min(1); - - DataSet> updatedComponentId = - verticesWithNewComponents - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .flatMap(new MinimumIdFilter(updatedElements)); - - List> result = - iteration.closeWith(updatedComponentId, updatedComponentId).collect(); - Collections.sort(result, new TestBaseUtils.TupleComparator>()); - - assertEquals(expectedResult, result); - } - - @Test - public void testParameterizableAggregator() throws Exception { - - final int maxIterations = 5; - final String aggregatorName = "elements.in.component.aggregator"; - final long componentId = 1L; - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> initialSolutionSet = env.fromCollection(verticesInput); - DataSet> edges = env.fromCollection(edgesInput); - - IterativeDataSet> iteration = initialSolutionSet.iterate(maxIterations); - - // register the aggregator - iteration.registerAggregator( - aggregatorName, new LongSumAggregatorWithParameter(componentId)); - - DataSet> verticesWithNewComponents = - iteration - .join(edges) - .where(0) - .equalTo(0) - .with(new NeighborWithComponentIDJoin()) - .groupBy(0) - .min(1); - - DataSet> updatedComponentId = - verticesWithNewComponents - .join(iteration) - .where(0) - .equalTo(0) - .flatMap(new MinimumIdFilterCounting(aggregatorName)); - - List> result = iteration.closeWith(updatedComponentId).collect(); - - Collections.sort(result, new TestBaseUtils.TupleComparator>()); - - List> expectedResult = - Arrays.asList( - new Tuple2<>(1L, 1L), - new Tuple2<>(2L, 1L), - new Tuple2<>(3L, 1L), - new Tuple2<>(4L, 1L), - new Tuple2<>(5L, 1L), - new Tuple2<>(6L, 1L), - new Tuple2<>(7L, 7L), - new Tuple2<>(8L, 7L), - new Tuple2<>(9L, 7L)); - - // check program result - assertEquals(expectedResult, result); - - // check aggregators - long[] aggrValues = MinimumIdFilterCounting.aggr_value; - - // note that position 0 has the end result from superstep 1, retrieved at the start of - // iteration 2 - // position one as superstep 2, retrieved at the start of iteration 3. - // the result from iteration 5 is not available, because no iteration 6 happens - assertEquals(3, aggrValues[0]); - assertEquals(4, aggrValues[1]); - assertEquals(5, aggrValues[2]); - assertEquals(6, aggrValues[3]); - } - - // ------------------------------------------------------------------------ - // Test Functions - // ------------------------------------------------------------------------ - - private static final class NeighborWithComponentIDJoin - extends RichJoinFunction, Tuple2, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 join( - Tuple2 vertexWithCompId, Tuple2 edge) { - vertexWithCompId.f0 = edge.f1; - return vertexWithCompId; - } - } - - private static class MinimumIdFilter - extends RichFlatMapFunction< - Tuple2, Tuple2>, Tuple2> { - - private final String aggName; - private LongSumAggregator aggr; - - public MinimumIdFilter(String aggName) { - this.aggName = aggName; - } - - @Override - public void open(OpenContext openContext) { - aggr = getIterationRuntimeContext().getIterationAggregator(aggName); - } - - @Override - public void flatMap( - Tuple2, Tuple2> vertexWithNewAndOldId, - Collector> out) { - - if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) { - out.collect(vertexWithNewAndOldId.f0); - aggr.aggregate(1L); - } else { - out.collect(vertexWithNewAndOldId.f1); - } - } - } - - private static final class MinimumIdFilterCounting - extends RichFlatMapFunction< - Tuple2, Tuple2>, Tuple2> { - - private static final long[] aggr_value = new long[5]; - - private final String aggName; - private LongSumAggregatorWithParameter aggr; - - public MinimumIdFilterCounting(String aggName) { - this.aggName = aggName; - } - - @Override - public void open(OpenContext openContext) { - final int superstep = getIterationRuntimeContext().getSuperstepNumber(); - - aggr = getIterationRuntimeContext().getIterationAggregator(aggName); - - if (superstep > 1 - && getIterationRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { - LongValue val = getIterationRuntimeContext().getPreviousIterationAggregate(aggName); - aggr_value[superstep - 2] = val.getValue(); - } - } - - @Override - public void flatMap( - Tuple2, Tuple2> vertexWithNewAndOldId, - Collector> out) { - - if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) { - out.collect(vertexWithNewAndOldId.f0); - if (vertexWithNewAndOldId.f0.f1 == aggr.getComponentId()) { - aggr.aggregate(1L); - } - } else { - out.collect(vertexWithNewAndOldId.f1); - if (vertexWithNewAndOldId.f1.f1 == aggr.getComponentId()) { - aggr.aggregate(1L); - } - } - } - } - - /** A Convergence Criterion with one parameter. */ - private static class UpdatedElementsConvergenceCriterion - implements ConvergenceCriterion { - - private final long threshold; - - public UpdatedElementsConvergenceCriterion(long uThreshold) { - this.threshold = uThreshold; - } - - @Override - public boolean isConverged(int iteration, LongValue value) { - return value.getValue() < this.threshold; - } - } - - private static final class LongSumAggregatorWithParameter extends LongSumAggregator { - - private long componentId; - - public LongSumAggregatorWithParameter(long compId) { - this.componentId = compId; - } - - public long getComponentId() { - return this.componentId; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java deleted file mode 100644 index 715cda4ffb0bd..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java +++ /dev/null @@ -1,529 +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.iterative.aggregators; - -import org.apache.flink.api.common.aggregators.ConvergenceCriterion; -import org.apache.flink.api.common.aggregators.LongSumAggregator; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -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.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.types.LongValue; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileReader; -import java.io.FileWriter; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; -import java.util.UUID; - -import static org.junit.Assert.assertEquals; - -/** Test the functionality of aggregators in bulk and delta iterative cases. */ -@RunWith(Parameterized.class) -public class AggregatorsITCase extends MultipleProgramsTestBaseJUnit4 { - - private static final int MAX_ITERATIONS = 20; - private static final int parallelism = 2; - private static final String NEGATIVE_ELEMENTS_AGGR = "count.negative.elements"; - - public AggregatorsITCase(TestExecutionMode mode) { - super(mode); - } - - @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); - - @Test - public void testDistributedCacheWithIterations() throws Exception { - final String testString = "Et tu, Brute?"; - final String testName = "testing_caesar"; - - final File folder = tempFolder.newFolder(); - final File resultFile = new File(folder, UUID.randomUUID().toString()); - - String testPath = resultFile.toString(); - String resultPath = resultFile.toURI().toString(); - - File tempFile = new File(testPath); - try (FileWriter writer = new FileWriter(tempFile)) { - writer.write(testString); - } - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.registerCachedFile(resultPath, testName); - - IterativeDataSet solution = env.fromElements(1L).iterate(2); - solution.closeWith( - env.generateSequence(1, 2) - .filter( - new RichFilterFunction() { - @Override - public void open(OpenContext openContext) - throws Exception { - File file = - getRuntimeContext() - .getDistributedCache() - .getFile(testName); - BufferedReader reader = - new BufferedReader(new FileReader(file)); - String output = reader.readLine(); - reader.close(); - assertEquals(output, testString); - } - - @Override - public boolean filter(Long value) throws Exception { - return false; - } - }) - .withBroadcastSet(solution, "SOLUTION")) - .output(new DiscardingOutputFormat()); - env.execute(); - } - - @Test - public void testAggregatorWithoutParameterForIterate() throws Exception { - /* - * Test aggregator without parameter for iterate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet initialSolutionSet = CollectionDataSets.getIntegerDataSet(env); - IterativeDataSet iteration = initialSolutionSet.iterate(MAX_ITERATIONS); - - // register aggregator - LongSumAggregator aggr = new LongSumAggregator(); - iteration.registerAggregator(NEGATIVE_ELEMENTS_AGGR, aggr); - - // register convergence criterion - iteration.registerAggregationConvergenceCriterion( - NEGATIVE_ELEMENTS_AGGR, aggr, new NegativeElementsConvergenceCriterion()); - - DataSet updatedDs = iteration.map(new SubtractOneMap()); - List result = iteration.closeWith(updatedDs).collect(); - Collections.sort(result); - - List expected = Arrays.asList(-3, -2, -2, -1, -1, -1, 0, 0, 0, 0, 1, 1, 1, 1, 1); - - assertEquals(expected, result); - } - - @Test - public void testAggregatorWithParameterForIterate() throws Exception { - /* - * Test aggregator with parameter for iterate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet initialSolutionSet = CollectionDataSets.getIntegerDataSet(env); - IterativeDataSet iteration = initialSolutionSet.iterate(MAX_ITERATIONS); - - // register aggregator - LongSumAggregatorWithParameter aggr = new LongSumAggregatorWithParameter(0); - iteration.registerAggregator(NEGATIVE_ELEMENTS_AGGR, aggr); - - // register convergence criterion - iteration.registerAggregationConvergenceCriterion( - NEGATIVE_ELEMENTS_AGGR, aggr, new NegativeElementsConvergenceCriterion()); - - DataSet updatedDs = iteration.map(new SubtractOneMapWithParam()); - List result = iteration.closeWith(updatedDs).collect(); - Collections.sort(result); - - List expected = Arrays.asList(-3, -2, -2, -1, -1, -1, 0, 0, 0, 0, 1, 1, 1, 1, 1); - - assertEquals(expected, result); - } - - @Test - public void testConvergenceCriterionWithParameterForIterate() throws Exception { - /* - * Test convergence criterion with parameter for iterate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet initialSolutionSet = CollectionDataSets.getIntegerDataSet(env); - IterativeDataSet iteration = initialSolutionSet.iterate(MAX_ITERATIONS); - - // register aggregator - LongSumAggregator aggr = new LongSumAggregator(); - iteration.registerAggregator(NEGATIVE_ELEMENTS_AGGR, aggr); - - // register convergence criterion - iteration.registerAggregationConvergenceCriterion( - NEGATIVE_ELEMENTS_AGGR, aggr, new NegativeElementsConvergenceCriterionWithParam(3)); - - DataSet updatedDs = iteration.map(new SubtractOneMap()); - List result = iteration.closeWith(updatedDs).collect(); - Collections.sort(result); - - List expected = Arrays.asList(-3, -2, -2, -1, -1, -1, 0, 0, 0, 0, 1, 1, 1, 1, 1); - - assertEquals(expected, result); - } - - @Test - public void testAggregatorWithoutParameterForIterateDelta() throws Exception { - /* - * Test aggregator without parameter for iterateDelta - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> initialSolutionSet = - CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap()); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialSolutionSet, MAX_ITERATIONS, 0); - - // register aggregator - LongSumAggregator aggr = new LongSumAggregator(); - iteration.registerAggregator(NEGATIVE_ELEMENTS_AGGR, aggr); - - DataSet> updatedDs = - iteration.getWorkset().map(new AggregateMapDelta()); - - DataSet> newElements = - updatedDs - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .flatMap(new UpdateFilter()); - - DataSet> iterationRes = - iteration.closeWith(newElements, newElements); - List result = iterationRes.map(new ProjectSecondMapper()).collect(); - Collections.sort(result); - - List expected = Arrays.asList(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5); - - assertEquals(expected, result); - } - - @Test - public void testAggregatorWithParameterForIterateDelta() throws Exception { - /* - * Test aggregator with parameter for iterateDelta - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> initialSolutionSet = - CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap()); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialSolutionSet, MAX_ITERATIONS, 0); - - // register aggregator - LongSumAggregator aggr = new LongSumAggregatorWithParameter(4); - iteration.registerAggregator(NEGATIVE_ELEMENTS_AGGR, aggr); - - DataSet> updatedDs = - iteration.getWorkset().map(new AggregateMapDelta()); - - DataSet> newElements = - updatedDs - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .flatMap(new UpdateFilter()); - - DataSet> iterationRes = - iteration.closeWith(newElements, newElements); - List result = iterationRes.map(new ProjectSecondMapper()).collect(); - Collections.sort(result); - - List expected = Arrays.asList(1, 2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5, 5); - - assertEquals(result, expected); - } - - @Test - public void testConvergenceCriterionWithParameterForIterateDelta() throws Exception { - /* - * Test convergence criterion with parameter for iterate delta - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - - DataSet> initialSolutionSet = - CollectionDataSets.getIntegerDataSet(env).map(new TupleMakerMap()); - - DeltaIteration, Tuple2> iteration = - initialSolutionSet.iterateDelta(initialSolutionSet, MAX_ITERATIONS, 0); - - // register aggregator - LongSumAggregator aggr = new LongSumAggregator(); - iteration.registerAggregator(NEGATIVE_ELEMENTS_AGGR, aggr); - - // register convergence criterion - iteration.registerAggregationConvergenceCriterion( - NEGATIVE_ELEMENTS_AGGR, aggr, new NegativeElementsConvergenceCriterionWithParam(3)); - - DataSet> updatedDs = - iteration.getWorkset().map(new AggregateAndSubtractOneDelta()); - - DataSet> newElements = - updatedDs.join(iteration.getSolutionSet()).where(0).equalTo(0).projectFirst(0, 1); - - DataSet> iterationRes = - iteration.closeWith(newElements, newElements); - List result = iterationRes.map(new ProjectSecondMapper()).collect(); - Collections.sort(result); - - List expected = Arrays.asList(-3, -2, -2, -1, -1, -1, 0, 0, 0, 0, 1, 1, 1, 1, 1); - - assertEquals(expected, result); - } - - @SuppressWarnings("serial") - private static final class NegativeElementsConvergenceCriterion - implements ConvergenceCriterion { - - @Override - public boolean isConverged(int iteration, LongValue value) { - return value.getValue() > 3; - } - } - - @SuppressWarnings("serial") - private static final class NegativeElementsConvergenceCriterionWithParam - implements ConvergenceCriterion { - - private int value; - - public NegativeElementsConvergenceCriterionWithParam(int val) { - this.value = val; - } - - public int getValue() { - return this.value; - } - - @Override - public boolean isConverged(int iteration, LongValue value) { - return value.getValue() > this.value; - } - } - - @SuppressWarnings("serial") - private static final class SubtractOneMap extends RichMapFunction { - - private LongSumAggregator aggr; - - @Override - public void open(OpenContext openContext) { - - aggr = getIterationRuntimeContext().getIterationAggregator(NEGATIVE_ELEMENTS_AGGR); - } - - @Override - public Integer map(Integer value) { - Integer newValue = value - 1; - // count negative numbers - if (newValue < 0) { - aggr.aggregate(1L); - } - return newValue; - } - } - - @SuppressWarnings("serial") - private static final class SubtractOneMapWithParam extends RichMapFunction { - - private LongSumAggregatorWithParameter aggr; - - @Override - public void open(OpenContext openContext) { - aggr = getIterationRuntimeContext().getIterationAggregator(NEGATIVE_ELEMENTS_AGGR); - } - - @Override - public Integer map(Integer value) { - Integer newValue = value - 1; - // count numbers less than the aggregator parameter - if (newValue < aggr.getValue()) { - aggr.aggregate(1L); - } - return newValue; - } - } - - @SuppressWarnings("serial") - private static class LongSumAggregatorWithParameter extends LongSumAggregator { - - private int value; - - public LongSumAggregatorWithParameter(int val) { - this.value = val; - } - - public int getValue() { - return this.value; - } - } - - @SuppressWarnings("serial") - private static final class TupleMakerMap - extends RichMapFunction> { - - private Random rnd; - - @Override - public void open(OpenContext openContext) { - rnd = - new Random( - 0xC0FFEBADBEEFDEADL - + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask()); - } - - @Override - public Tuple2 map(Integer value) { - Integer nodeId = rnd.nextInt(100000); - return new Tuple2<>(nodeId, value); - } - } - - @SuppressWarnings("serial") - private static final class AggregateMapDelta - extends RichMapFunction, Tuple2> { - - private LongSumAggregator aggr; - private LongValue previousAggr; - private int superstep; - - @Override - public void open(OpenContext openContext) { - aggr = getIterationRuntimeContext().getIterationAggregator(NEGATIVE_ELEMENTS_AGGR); - superstep = getIterationRuntimeContext().getSuperstepNumber(); - - if (superstep > 1) { - previousAggr = - getIterationRuntimeContext() - .getPreviousIterationAggregate(NEGATIVE_ELEMENTS_AGGR); - // check previous aggregator value - Assert.assertEquals(superstep - 1, previousAggr.getValue()); - } - } - - @Override - public Tuple2 map(Tuple2 value) { - // count the elements that are equal to the superstep number - if (value.f1 == superstep) { - aggr.aggregate(1L); - } - return value; - } - } - - @SuppressWarnings("serial") - private static final class UpdateFilter - extends RichFlatMapFunction< - Tuple2, Tuple2>, - Tuple2> { - - private int superstep; - - @Override - public void open(OpenContext openContext) { - superstep = getIterationRuntimeContext().getSuperstepNumber(); - } - - @Override - public void flatMap( - Tuple2, Tuple2> value, - Collector> out) { - - if (value.f0.f1 > superstep) { - out.collect(value.f0); - } - } - } - - @SuppressWarnings("serial") - private static final class ProjectSecondMapper - extends RichMapFunction, Integer> { - - @Override - public Integer map(Tuple2 value) { - return value.f1; - } - } - - @SuppressWarnings("serial") - private static final class AggregateAndSubtractOneDelta - extends RichMapFunction, Tuple2> { - - private LongSumAggregator aggr; - private LongValue previousAggr; - private int superstep; - - @Override - public void open(OpenContext openContext) { - aggr = getIterationRuntimeContext().getIterationAggregator(NEGATIVE_ELEMENTS_AGGR); - superstep = getIterationRuntimeContext().getSuperstepNumber(); - - if (superstep > 1) { - previousAggr = - getIterationRuntimeContext() - .getPreviousIterationAggregate(NEGATIVE_ELEMENTS_AGGR); - // check previous aggregator value - Assert.assertEquals(superstep - 1, previousAggr.getValue()); - } - } - - @Override - public Tuple2 map(Tuple2 value) { - // count the ones - if (value.f1 == 1) { - aggr.aggregate(1L); - } - value.f1--; - return value; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java b/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java index f9262dc5a611a..891b4253d283f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java @@ -23,11 +23,13 @@ import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -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.api.java.typeutils.TupleTypeInfo; +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.types.IntValue; +import org.apache.flink.util.CollectionUtil; import org.junit.Assert; import org.slf4j.Logger; @@ -70,7 +72,7 @@ public static void main(String[] args) throws Exception { public void run() throws Exception { LOG.info("Random seed = {}", RANDOM_SEED); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); for (int parallelism = MAX_PARALLELISM; parallelism > 0; parallelism--) { LOG.info("Parallelism = {}", parallelism); @@ -80,13 +82,12 @@ public void run() throws Exception { testReduce(env); testGroupedReduce(env); testJoin(env); - testCross(env); } } // -------------------------------------------------------------------------------------------- - public void testReduce(ExecutionEnvironment env) throws Exception { + public void testReduce(StreamExecutionEnvironment env) throws Exception { /* * Test ChainedAllReduceDriver */ @@ -96,12 +97,22 @@ public void testReduce(ExecutionEnvironment env) throws Exception { env.getConfig().enableObjectReuse(); Tuple2 enabledResult = - getDataSet(env).reduce(new OverwriteObjectsReduce(false)).collect().get(0); + CollectionUtil.iteratorToList( + getDataStream(env) + .windowAll(GlobalWindows.create()) + .reduce(new OverwriteObjectsReduce(false)) + .executeAndCollect()) + .get(0); env.getConfig().disableObjectReuse(); Tuple2 disabledResult = - getDataSet(env).reduce(new OverwriteObjectsReduce(false)).collect().get(0); + CollectionUtil.iteratorToList( + getDataStream(env) + .windowAll(GlobalWindows.create()) + .reduce(new OverwriteObjectsReduce(false)) + .executeAndCollect()) + .get(0); Assert.assertEquals(NUMBER_OF_ELEMENTS, enabledResult.f1.getValue()); Assert.assertEquals(NUMBER_OF_ELEMENTS, disabledResult.f1.getValue()); @@ -109,7 +120,7 @@ public void testReduce(ExecutionEnvironment env) throws Exception { Assert.assertEquals(disabledResult, enabledResult); } - public void testGroupedReduce(ExecutionEnvironment env) throws Exception { + public void testGroupedReduce(StreamExecutionEnvironment env) throws Exception { /* * Test ReduceCombineDriver and ReduceDriver */ @@ -119,14 +130,22 @@ public void testGroupedReduce(ExecutionEnvironment env) throws Exception { env.getConfig().enableObjectReuse(); List> enabledResult = - getDataSet(env).groupBy(0).reduce(new OverwriteObjectsReduce(true)).collect(); + CollectionUtil.iteratorToList( + getDataStream(env) + .keyBy(x -> x.f0) + .reduce(new OverwriteObjectsReduce(true)) + .executeAndCollect()); Collections.sort(enabledResult, comparator); env.getConfig().disableObjectReuse(); List> disabledResult = - getDataSet(env).groupBy(0).reduce(new OverwriteObjectsReduce(true)).collect(); + CollectionUtil.iteratorToList( + getDataStream(env) + .keyBy(x -> x.f0) + .reduce(new OverwriteObjectsReduce(true)) + .executeAndCollect()); Collections.sort(disabledResult, comparator); @@ -149,7 +168,7 @@ public Tuple2 reduce( // -------------------------------------------------------------------------------------------- - public void testJoin(ExecutionEnvironment env) throws Exception { + public void testJoin(StreamExecutionEnvironment env) throws Exception { /* * Test JoinDriver, LeftOuterJoinDriver, RightOuterJoinDriver, and FullOuterJoinDriver */ @@ -170,125 +189,32 @@ public void testJoin(ExecutionEnvironment env) throws Exception { env.getConfig().enableObjectReuse(); enabledResult = - getDataSet(env) - .join(getDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); + CollectionUtil.iteratorToList( + getDataStream(env) + .join(getDataStream(env)) + .where(x -> x.f0) + .equalTo(x -> x.f0) + .window(GlobalWindows.create()) + .apply(new OverwriteObjectsJoin()) + .executeAndCollect()); Collections.sort(enabledResult, comparator); env.getConfig().disableObjectReuse(); disabledResult = - getDataSet(env) - .join(getDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); + CollectionUtil.iteratorToList( + getDataStream(env) + .join(getDataStream(env)) + .where(x -> x.f0) + .equalTo(x -> x.f0) + .window(GlobalWindows.create()) + .apply(new OverwriteObjectsJoin()) + .executeAndCollect()); Collections.sort(disabledResult, comparator); Assert.assertEquals("JoinHint=" + joinHint, disabledResult, enabledResult); - - // Left outer join - - if (joinHint != JoinHint.BROADCAST_HASH_FIRST) { - LOG.info("Testing left outer join with JoinHint = {}", joinHint); - - env.getConfig().enableObjectReuse(); - - enabledResult = - getDataSet(env) - .leftOuterJoin(getFilteredDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); - - Collections.sort(enabledResult, comparator); - - env.getConfig().disableObjectReuse(); - - disabledResult = - getDataSet(env) - .leftOuterJoin(getFilteredDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); - - Collections.sort(disabledResult, comparator); - - Assert.assertThat("JoinHint=" + joinHint, disabledResult, is(enabledResult)); - } - - // Right outer join - - if (joinHint != JoinHint.BROADCAST_HASH_SECOND) { - LOG.info("Testing right outer join with JoinHint = {}", joinHint); - - env.getConfig().enableObjectReuse(); - - enabledResult = - getDataSet(env) - .rightOuterJoin(getFilteredDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); - - Collections.sort(enabledResult, comparator); - - env.getConfig().disableObjectReuse(); - - disabledResult = - getDataSet(env) - .rightOuterJoin(getFilteredDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); - - Collections.sort(disabledResult, comparator); - - Assert.assertThat("JoinHint=" + joinHint, disabledResult, is(enabledResult)); - } - - // Full outer join - - if (joinHint != JoinHint.BROADCAST_HASH_FIRST - && joinHint != JoinHint.BROADCAST_HASH_SECOND) { - LOG.info("Testing full outer join with JoinHint = {}", joinHint); - - env.getConfig().enableObjectReuse(); - - enabledResult = - getDataSet(env) - .fullOuterJoin(getFilteredDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); - - Collections.sort(enabledResult, comparator); - - env.getConfig().disableObjectReuse(); - - disabledResult = - getDataSet(env) - .fullOuterJoin(getFilteredDataSet(env), joinHint) - .where(0) - .equalTo(0) - .with(new OverwriteObjectsJoin()) - .collect(); - - Collections.sort(disabledResult, comparator); - - Assert.assertThat("JoinHint=" + joinHint, disabledResult, is(enabledResult)); - } } } @@ -306,49 +232,6 @@ public Tuple2 join( } } - // -------------------------------------------------------------------------------------------- - - public void testCross(ExecutionEnvironment env) throws Exception { - /* - * Test CrossDriver - */ - - LOG.info("Testing cross"); - - DataSet> small = getDataSet(env, 100, 20); - DataSet> large = getDataSet(env, 10000, 2000); - - // test NESTEDLOOP_BLOCKED_OUTER_FIRST and NESTEDLOOP_BLOCKED_OUTER_SECOND with object reuse - // enabled - - env.getConfig().enableObjectReuse(); - - List> enabledResultWithHuge = - small.crossWithHuge(large).with(new OverwriteObjectsCross()).collect(); - - List> enabledResultWithTiny = - small.crossWithTiny(large).with(new OverwriteObjectsCross()).collect(); - - Assert.assertThat(enabledResultWithHuge, is(enabledResultWithTiny)); - - // test NESTEDLOOP_BLOCKED_OUTER_FIRST and NESTEDLOOP_BLOCKED_OUTER_SECOND with object reuse - // disabled - - env.getConfig().disableObjectReuse(); - - List> disabledResultWithHuge = - small.crossWithHuge(large).with(new OverwriteObjectsCross()).collect(); - - List> disabledResultWithTiny = - small.crossWithTiny(large).with(new OverwriteObjectsCross()).collect(); - - Assert.assertThat(disabledResultWithHuge, is(disabledResultWithTiny)); - - // verify match between object reuse enabled and disabled - Assert.assertThat(disabledResultWithHuge, is(enabledResultWithHuge)); - Assert.assertThat(disabledResultWithTiny, is(enabledResultWithTiny)); - } - private class OverwriteObjectsCross implements CrossFunction< Tuple2, @@ -365,20 +248,21 @@ public Tuple2 cross( // -------------------------------------------------------------------------------------------- - private DataSet> getDataSet( - ExecutionEnvironment env, int numberOfElements, int keyRange) { + private DataStream> getDataStream( + StreamExecutionEnvironment env, int numberOfElements, int keyRange) { return env.fromCollection( new TupleIntValueIntValueIterator(numberOfElements, keyRange), TupleTypeInfo.>getBasicAndBasicValueTupleTypeInfo( IntValue.class, IntValue.class)); } - private DataSet> getDataSet(ExecutionEnvironment env) { - return getDataSet(env, NUMBER_OF_ELEMENTS, KEY_RANGE); + private DataStream> getDataStream(StreamExecutionEnvironment env) { + return getDataStream(env, NUMBER_OF_ELEMENTS, KEY_RANGE); } - private DataSet> getFilteredDataSet(ExecutionEnvironment env) { - return getDataSet(env) + private DataStream> getFilteredDataStream( + StreamExecutionEnvironment env) { + return getDataStream(env) .filter( new FilterFunction>() { @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java b/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java index e8f29df8095d5..c3d664750c19d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java @@ -21,10 +21,12 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.SplittableIterator; import java.io.Serializable; @@ -102,20 +104,19 @@ private static > void testReducePerformance( boolean print) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().enableObjectReuse(); @SuppressWarnings("unchecked") - DataSet output = + DataStream output = env.fromParallelCollection( new SplittableRandomIterator(numRecords, iterator), typeInfo) - .groupBy("0") - .reduce(new SumReducer()) - .setCombineHint(hint); + .keyBy(x -> ((Tuple) x).getField(0)) + .reduce(new SumReducer()); long start = System.currentTimeMillis(); - System.out.println(output.count()); + System.out.println(CollectionUtil.iteratorToList(output.executeAndCollect()).size()); long end = System.currentTimeMillis(); if (print) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java index b468f16608150..292d4c43c83f7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.junit.Assert; @@ -33,16 +33,14 @@ public class CustomPartitioningITCase extends JavaProgramTestBaseJUnit4 { @Override protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - if (!isCollectionExecution()) { - Assert.assertTrue(env.getParallelism() > 1); - } + Assert.assertTrue(env.getParallelism() > 1); - env.generateSequence(1, 1000) + env.fromSequence(1, 1000) .partitionCustom(new AllZeroPartitioner(), new IdKeySelector()) .map(new FailExceptInPartitionZeroMapper()) - .output(new DiscardingOutputFormat()); + .sinkTo(new DiscardingSink<>()); env.execute(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java index 639f7cdaa84e2..e74251997c83b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java @@ -19,8 +19,6 @@ package org.apache.flink.test.misc; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; @@ -29,6 +27,8 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Value; import org.apache.flink.util.TestLogger; @@ -70,10 +70,10 @@ public static Configuration getConfiguration() { @Test public void testIncorrectSerializer1() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARLLELISM); - env.generateSequence(1, 10 * PARLLELISM) + env.fromSequence(1, 10 * PARLLELISM) .map( new MapFunction() { @Override @@ -82,7 +82,7 @@ public ConsumesTooMuch map(Long value) throws Exception { } }) .rebalance() - .output(new DiscardingOutputFormat()); + .sinkTo(new DiscardingSink<>()); env.execute(); } catch (JobExecutionException e) { @@ -98,10 +98,10 @@ public ConsumesTooMuch map(Long value) throws Exception { @Test public void testIncorrectSerializer2() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARLLELISM); - env.generateSequence(1, 10 * PARLLELISM) + env.fromSequence(1, 10 * PARLLELISM) .map( new MapFunction() { @Override @@ -110,11 +110,12 @@ public ConsumesTooMuchSpanning map(Long value) throws Exception { } }) .rebalance() - .output(new DiscardingOutputFormat()); + .sinkTo(new DiscardingSink<>()); env.execute(); } catch (JobExecutionException e) { - Optional rootCause = findThrowable(e, IOException.class); + Optional rootCause = + findThrowable(e, t -> t.getMessage().contains("broken serialization")); assertTrue(rootCause.isPresent()); assertTrue(rootCause.get().getMessage().contains("broken serialization")); } catch (Exception e) { @@ -126,10 +127,10 @@ public ConsumesTooMuchSpanning map(Long value) throws Exception { @Test public void testIncorrectSerializer3() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARLLELISM); - env.generateSequence(1, 10 * PARLLELISM) + env.fromSequence(1, 10 * PARLLELISM) .map( new MapFunction() { @Override @@ -138,7 +139,7 @@ public ConsumesTooLittle map(Long value) throws Exception { } }) .rebalance() - .output(new DiscardingOutputFormat()); + .sinkTo(new DiscardingSink<>()); env.execute(); } catch (JobExecutionException e) { @@ -154,10 +155,10 @@ public ConsumesTooLittle map(Long value) throws Exception { @Test public void testIncorrectSerializer4() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARLLELISM); - env.generateSequence(1, 10 * PARLLELISM) + env.fromSequence(1, 10 * PARLLELISM) .map( new MapFunction() { @Override @@ -166,7 +167,7 @@ public ConsumesTooLittleSpanning map(Long value) throws Exception { } }) .rebalance() - .output(new DiscardingOutputFormat()); + .sinkTo(new DiscardingSink<>()); env.execute(); } catch (ProgramInvocationException e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java b/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java index 9bcb1948c1d41..e9979e69209f2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java @@ -19,9 +19,9 @@ package org.apache.flink.test.misc; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.test.operators.util.CollectionDataSets; +import org.apache.flink.test.operators.util.CollectionDataStreams; +import org.apache.flink.util.Utils; import org.junit.Assert; import org.junit.Test; @@ -34,10 +34,10 @@ public class GenericTypeInfoTest { @Test public void testSerializerTree() { @SuppressWarnings("unchecked") - TypeInformation ti = - (TypeInformation) + TypeInformation ti = + (TypeInformation) TypeExtractor.createTypeInfo( - CollectionDataSets.PojoWithCollectionGeneric.class); + CollectionDataStreams.PojoWithCollectionGeneric.class); final String serTree = Utils.getSerializerTree(ti) @@ -64,9 +64,9 @@ public void testSerializerTree() { + " scalaBigInt:scala.math.BigInt\n" + " bigInteger:java.math.BigInteger\n" + " mixed:java.util.List\n" - + " makeMeGeneric:org.apache.flink.test.operators.util.CollectionDataSets$PojoWithDateAndEnum\n" + + " makeMeGeneric:org.apache.flink.test.operators.util.CollectionDataStreams$PojoWithDateAndEnum\n" + " group:java.lang.String\n" + " date:java.util.Date\n" - + " cat:org.apache.flink.test.operators.util.CollectionDataSets$Category (is enum)\n")); + + " cat:org.apache.flink.test.operators.util.CollectionDataStreams$Category (is enum)\n")); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 0f3626b174a09..6febe2ea47310 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -23,12 +23,14 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; -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.core.fs.Path; import org.apache.flink.runtime.client.JobExecutionException; 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.test.util.MiniClusterWithClientResource; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -63,11 +65,11 @@ public class MiscellaneousIssuesITCase extends TestLogger { @Test public void testNullValues() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); - DataSet data = - env.fromElements("hallo") + DataStream data = + env.fromData("hallo") .map( new MapFunction() { @Override @@ -75,7 +77,10 @@ public String map(String value) throws Exception { return null; } }); - data.writeAsText("/tmp/myTest", FileSystem.WriteMode.OVERWRITE); + data.sinkTo( + FileSink.forRowFormat( + new Path("/tmp/myTest"), new SimpleStringEncoder()) + .build()); try { env.execute(); @@ -92,12 +97,12 @@ public String map(String value) throws Exception { @Test public void testDisjointDataflows() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(5); // generate two different flows - env.generateSequence(1, 10).output(new DiscardingOutputFormat()); - env.generateSequence(1, 10).output(new DiscardingOutputFormat()); + env.fromSequence(1, 10).sinkTo(new DiscardingSink<>()); + env.fromSequence(1, 10).sinkTo(new DiscardingSink<>()); } catch (Exception e) { e.printStackTrace(); @@ -111,10 +116,10 @@ public void testAccumulatorsAfterNoOp() { final String accName = "test_accumulator"; try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(6); - env.generateSequence(1, 1000000) + env.fromSequence(1, 1000000) .rebalance() .flatMap( new RichFlatMapFunction() { @@ -131,7 +136,7 @@ public void flatMap(Long value, Collector out) { counter.add(1L); } }) - .output(new DiscardingOutputFormat()); + .sinkTo(new DiscardingSink<>()); JobExecutionResult result = env.execute(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java deleted file mode 100644 index afd02301bb6c3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java +++ /dev/null @@ -1,173 +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.misc; - -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.examples.java.clustering.KMeans; -import org.apache.flink.examples.java.clustering.util.KMeansData; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.examples.java.graph.util.ConnectedComponentsData; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.TestLogger; - -import org.junit.ClassRule; -import org.junit.Test; - -import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Test that runs an iterative job after a failure in another iterative job. This test validates - * that task slots in co-location constraints are properly freed in the presence of failures. - */ -public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger { - - private static final int PARALLELISM = 4; - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setConfiguration(getConfiguration()) - .setNumberTaskManagers(2) - .setNumberSlotsPerTaskManager(2) - .build()); - - private static Configuration getConfiguration() { - Configuration config = new Configuration(); - config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("20m")); - config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.ofMebiBytes(3L)); - config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.ofMebiBytes(3L)); - return config; - } - - @Test - public void testSuccessfulProgramAfterFailure() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - runConnectedComponents(env); - - try { - runKMeans(env); - fail("This program execution should have failed."); - } catch (JobExecutionException e) { - assertTrue( - findThrowableWithMessage(e, "Insufficient number of network buffers") - .isPresent()); - } - - runConnectedComponents(env); - } - - private static void runConnectedComponents(ExecutionEnvironment env) throws Exception { - - env.setParallelism(PARALLELISM); - - // read vertex and edge data - DataSet vertices = ConnectedComponentsData.getDefaultVertexDataSet(env).rebalance(); - - DataSet> edges = - ConnectedComponentsData.getDefaultEdgeDataSet(env) - .rebalance() - .flatMap(new ConnectedComponents.UndirectEdge()); - - // assign the initial components (equal to the vertex id) - DataSet> verticesWithInitialId = - vertices.map(new ConnectedComponents.DuplicateValue()); - - // open a delta iteration - DeltaIteration, Tuple2> iteration = - verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0); - - // apply the step logic: join with the edges, select the minimum neighbor, - // update if the component of the candidate is smaller - DataSet> changes = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new ConnectedComponents.NeighborWithComponentIDJoin()) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new ConnectedComponents.ComponentIdFilter()); - - // close the delta iteration (delta and new workset are identical) - DataSet> result = iteration.closeWith(changes, changes); - - result.output(new DiscardingOutputFormat>()); - - env.execute(); - } - - private static void runKMeans(ExecutionEnvironment env) throws Exception { - - env.setParallelism(PARALLELISM); - - // get input data - DataSet points = KMeansData.getDefaultPointDataSet(env).rebalance(); - DataSet centroids = KMeansData.getDefaultCentroidDataSet(env).rebalance(); - - // set number of bulk iterations for KMeans algorithm - IterativeDataSet loop = centroids.iterate(20); - - // add some re-partitions to increase network buffer use - DataSet newCentroids = - points - // compute closest centroid for each point - .map(new KMeans.SelectNearestCenter()) - .withBroadcastSet(loop, "centroids") - .rebalance() - // count and sum point coordinates for each centroid - .map(new KMeans.CountAppender()) - .groupBy(0) - .reduce(new KMeans.CentroidAccumulator()) - // compute new centroids from point counts and coordinate sums - .rebalance() - .map(new KMeans.CentroidAverager()); - - // feed new centroids back into next iteration - DataSet finalCentroids = loop.closeWith(newCentroids); - - DataSet> clusteredPoints = - points - // assign points to final clusters - .map(new KMeans.SelectNearestCenter()) - .withBroadcastSet(finalCentroids, "centroids"); - - clusteredPoints.output(new DiscardingOutputFormat>()); - - env.execute("KMeans Example"); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java deleted file mode 100644 index 185f341588dca..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java +++ /dev/null @@ -1,172 +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.operators; - -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.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.ValueCollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for aggregations. */ -@RunWith(Parameterized.class) -public class AggregateITCase extends MultipleProgramsTestBaseJUnit4 { - - public AggregateITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testFullAggregate() throws Exception { - /* - * Full Aggregate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = - ds.aggregate(Aggregations.SUM, 0).and(Aggregations.MAX, 1).project(0, 1); - - List> result = aggregateDs.collect(); - - String expected = "231,6\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testFullAggregateOfMutableValueTypes() throws Exception { - /* - * Full Aggregate of mutable value types - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - ValueCollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = - ds.aggregate(Aggregations.SUM, 0).and(Aggregations.MAX, 1).project(0, 1); - - List> result = aggregateDs.collect(); - - String expected = "231,6\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testGroupedAggregate() throws Exception { - /* - * Grouped Aggregate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = - ds.groupBy(1).aggregate(Aggregations.SUM, 0).project(1, 0); - - List> result = aggregateDs.collect(); - - String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testGroupedAggregateOfMutableValueTypes() throws Exception { - /* - * Grouped Aggregate of mutable value types - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - ValueCollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = - ds.groupBy(1).aggregate(Aggregations.SUM, 0).project(1, 0); - - List> result = aggregateDs.collect(); - - String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedAggregate() throws Exception { - /* - * Nested Aggregate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = - ds.groupBy(1) - .aggregate(Aggregations.MIN, 0) - .aggregate(Aggregations.MIN, 0) - .project(0); - - List> result = aggregateDs.collect(); - - String expected = "1\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedAggregateOfMutableValueTypes() throws Exception { - /* - * Nested Aggregate of mutable value types - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - ValueCollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = - ds.groupBy(1) - .aggregate(Aggregations.MIN, 0) - .aggregate(Aggregations.MIN, 0) - .project(0); - - List> result = aggregateDs.collect(); - - String expected = "1\n"; - - compareResultAsTuples(result, expected); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java deleted file mode 100644 index 42cf043ecbc7c..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java +++ /dev/null @@ -1,128 +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.operators; - -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.operators.Order; -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.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.types.NullValue; -import org.apache.flink.util.Collector; - -import org.junit.Assert; - -/** Integration tests for {@link CoGroupFunction}. */ -@SuppressWarnings({"serial", "unchecked"}) -public class CoGroupGroupSortITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input1 = - env.fromElements( - new Tuple2(0L, 5L), - new Tuple2(0L, 4L), - new Tuple2(0L, 3L), - new Tuple2(0L, 2L), - new Tuple2(0L, 1L), - new Tuple2(1L, 10L), - new Tuple2(1L, 8L), - new Tuple2(1L, 9L), - new Tuple2(1L, 7L)); - - DataSet input2 = - env.fromElements( - new TestPojo(0L, 10L, 3L), - new TestPojo(0L, 8L, 3L), - new TestPojo(0L, 10L, 1L), - new TestPojo(0L, 9L, 0L), - new TestPojo(0L, 8L, 2L), - new TestPojo(0L, 8L, 4L), - new TestPojo(1L, 10L, 3L), - new TestPojo(1L, 8L, 3L), - new TestPojo(1L, 10L, 1L), - new TestPojo(1L, 9L, 0L), - new TestPojo(1L, 8L, 2L), - new TestPojo(1L, 8L, 4L)); - - input1.coGroup(input2) - .where(1) - .equalTo("b") - .sortFirstGroup(0, Order.DESCENDING) - .sortSecondGroup("c", Order.ASCENDING) - .sortSecondGroup("a", Order.DESCENDING) - .with(new ValidatingCoGroup()) - .output(new DiscardingOutputFormat()); - - env.execute(); - } - - private static class ValidatingCoGroup - implements CoGroupFunction, TestPojo, NullValue> { - - @Override - public void coGroup( - Iterable> first, - Iterable second, - Collector out) - throws Exception { - // validate the tuple input, field 1, descending - { - long lastValue = Long.MAX_VALUE; - - for (Tuple2 t : first) { - long current = t.f1; - Assert.assertTrue(current <= lastValue); - lastValue = current; - } - } - - // validate the pojo input - { - TestPojo lastValue = new TestPojo(Long.MAX_VALUE, 0, Long.MIN_VALUE); - - for (TestPojo current : second) { - Assert.assertTrue(current.c >= lastValue.c); - Assert.assertTrue(current.c != lastValue.c || current.a <= lastValue.a); - - lastValue = current; - } - } - } - } - - /** Test POJO. */ - public static class TestPojo implements Cloneable { - public long a; - public long b; - public long c; - - public TestPojo() {} - - public TestPojo(long a, long b, long c) { - this.a = a; - this.b = b; - this.c = c; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java deleted file mode 100644 index b03e7ac3eded0..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java +++ /dev/null @@ -1,1149 +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.operators; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichCoGroupFunction; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link CoGroupFunction} and {@link RichCoGroupFunction}. */ -@RunWith(Parameterized.class) -public class CoGroupITCase extends MultipleProgramsTestBaseJUnit4 { - - public CoGroupITCase(TestExecutionMode mode) { - super(mode); - } - - /* - * CoGroup on tuples with key field selector - */ - @Test - public void testCoGroupTuplesWithKeyFieldSelector() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> coGroupDs = - ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5CoGroup()); - - List> result = coGroupDs.collect(); - - String expected = "1,0\n" + "2,6\n" + "3,24\n" + "4,60\n" + "5,120\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupOnTwoCustomTypeInputsWithKeyExtractors() throws Exception { - /* - * CoGroup on two custom type inputs with key extractors - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet ds2 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet coGroupDs = - ds.coGroup(ds2) - .where(new KeySelector4()) - .equalTo(new KeySelector5()) - .with(new CustomTypeCoGroup()); - - List result = coGroupDs.collect(); - - String expected = - "1,0,test\n" - + "2,6,test\n" - + "3,24,test\n" - + "4,60,test\n" - + "5,120,test\n" - + "6,210,test\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector4 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - private static class KeySelector5 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testCorrectnessOfCoGroupIfUDFReturnsLeftInputObjects() throws Exception { - /* - * check correctness of cogroup if UDF returns left input objects - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - DataSet> coGroupDs = - ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple3ReturnLeft()); - - List> result = coGroupDs.collect(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCoGroupIfUDFReturnsRightInputObjects() throws Exception { - /* - * check correctness of cogroup if UDF returns right input objects - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> coGroupDs = - ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5ReturnRight()); - - List> result = coGroupDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "2,2,1,Hallo Welt,2\n" - + "2,3,2,Hallo Welt wie,1\n" - + "3,4,3,Hallo Welt wie gehts?,2\n" - + "3,5,4,ABC,2\n" - + "3,6,5,BCD,3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupWithBroadcastSet() throws Exception { - /* - * Reduce with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> coGroupDs = - ds.coGroup(ds2) - .where(0) - .equalTo(0) - .with(new Tuple5CoGroupBC()) - .withBroadcastSet(intDs, "ints"); - - List> result = coGroupDs.collect(); - - String expected = "1,0,55\n" + "2,6,55\n" + "3,24,55\n" + "4,60,55\n" + "5,120,55\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor() - throws Exception { - /* - * CoGroup on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet ds2 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet> coGroupDs = - ds.coGroup(ds2).where(2).equalTo(new KeySelector2()).with(new MixedCoGroup()); - - List> result = coGroupDs.collect(); - - String expected = - "0,1,test\n" - + "1,2,test\n" - + "2,5,test\n" - + "3,15,test\n" - + "4,33,test\n" - + "5,63,test\n" - + "6,109,test\n" - + "7,4,test\n" - + "8,4,test\n" - + "9,4,test\n" - + "10,5,test\n" - + "11,5,test\n" - + "12,5,test\n" - + "13,5,test\n" - + "14,5,test\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector2 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testCoGroupOnACustomTypeWithKeyExtractorAndATupleInputWithKeyFieldSelector() - throws Exception { - /* - * CoGroup on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet ds2 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet coGroupDs = - ds2.coGroup(ds).where(new KeySelector3()).equalTo(2).with(new MixedCoGroup2()); - - List result = coGroupDs.collect(); - - String expected = - "0,1,test\n" - + "1,2,test\n" - + "2,5,test\n" - + "3,15,test\n" - + "4,33,test\n" - + "5,63,test\n" - + "6,109,test\n" - + "7,4,test\n" - + "8,4,test\n" - + "9,4,test\n" - + "10,5,test\n" - + "11,5,test\n" - + "12,5,test\n" - + "13,5,test\n" - + "14,5,test\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector3 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testCoGroupWithMultipleKeyFieldsWithFieldSelector() throws Exception { - /* - * CoGroup with multiple key fields - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - - DataSet> coGrouped = - ds1.coGroup(ds2).where(0, 4).equalTo(0, 1).with(new Tuple5Tuple3CoGroup()); - - List> result = coGrouped.collect(); - - String expected = - "1,1,Hallo\n" - + "2,2,Hallo Welt\n" - + "3,2,Hallo Welt wie gehts?\n" - + "3,2,ABC\n" - + "5,3,HIJ\n" - + "5,3,IJK\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupWithMultipleKeyFieldsWithStaticClassKeyExtractor() throws Exception { - /* - * CoGroup with multiple key fields - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - - DataSet> coGrouped = - ds1.coGroup(ds2) - .where(new KeySelector7()) - .equalTo(new KeySelector8()) - .with(new Tuple5Tuple3CoGroup()); - - List> result = coGrouped.collect(); - - String expected = - "1,1,Hallo\n" - + "2,2,Hallo Welt\n" - + "3,2,Hallo Welt wie gehts?\n" - + "3,2,ABC\n" - + "5,3,HIJ\n" - + "5,3,IJK\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupWithMultipleKeyFieldsWithInnerClassKeyExtractorWithClosureCleaner() - throws Exception { - /* - * CoGroup with multiple key fields, test working closure cleaner for inner classes - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - - DataSet> coGrouped = - ds1.coGroup(ds2) - .where( - new KeySelector< - Tuple5, - Tuple2>() { - @Override - public Tuple2 getKey( - Tuple5 t) - throws Exception { - return new Tuple2(t.f0, t.f4); - } - }) - .equalTo( - new KeySelector< - Tuple3, Tuple2>() { - - @Override - public Tuple2 getKey( - Tuple3 t) { - return new Tuple2<>(t.f0, t.f1); - } - }) - .with( - new CoGroupFunction< - Tuple5, - Tuple3, - Tuple3>() { - @Override - public void coGroup( - Iterable> - first, - Iterable> second, - Collector> out) { - List strs = new ArrayList<>(); - - for (Tuple5 t : - first) { - strs.add(t.f3); - } - - for (Tuple3 t : second) { - for (String s : strs) { - out.collect( - new Tuple3( - t.f0, t.f1, s)); - } - } - } - }); - - List> result = coGrouped.collect(); - - String expected = - "1,1,Hallo\n" - + "2,2,Hallo Welt\n" - + "3,2,Hallo Welt wie gehts?\n" - + "3,2,ABC\n" - + "5,3,HIJ\n" - + "5,3,IJK\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupWithMultipleKeyFieldsWithInnerClassKeyExtractorWithoutClosureCleaner() - throws Exception { - /* - * CoGroup with multiple key fields, test that disabling closure cleaner leads to an exception when using inner - * classes. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().disableClosureCleaner(); - - DataSet> ds1 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - boolean correctExceptionTriggered = false; - try { - DataSet> coGrouped = - ds1.coGroup(ds2) - .where( - new KeySelector< - Tuple5, - Tuple2>() { - @Override - public Tuple2 getKey( - Tuple5 t) - throws Exception { - return new Tuple2(t.f0, t.f4); - } - }) - .equalTo( - new KeySelector< - Tuple3, - Tuple2>() { - - @Override - public Tuple2 getKey( - Tuple3 t) { - return new Tuple2(t.f0, t.f1); - } - }) - .with( - new CoGroupFunction< - Tuple5, - Tuple3, - Tuple3>() { - @Override - public void coGroup( - Iterable< - Tuple5< - Integer, - Long, - Integer, - String, - Long>> - first, - Iterable> second, - Collector> out) { - List strs = new ArrayList(); - - for (Tuple5 t : - first) { - strs.add(t.f3); - } - - for (Tuple3 t : second) { - for (String s : strs) { - out.collect( - new Tuple3( - t.f0, t.f1, s)); - } - } - } - }); - } catch (InvalidProgramException ex) { - correctExceptionTriggered = (ex.getCause() instanceof java.io.NotSerializableException); - } - Assert.assertTrue(correctExceptionTriggered); - } - - private static class KeySelector7 - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 t) { - return new Tuple2(t.f0, t.f4); - } - } - - private static class KeySelector8 - implements KeySelector, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple3 t) { - return new Tuple2(t.f0, t.f1); - } - } - - @Test - public void testCoGroupTwoCustomTypeInputsWithExpressionKeys() throws Exception { - /* - * CoGroup on two custom type inputs using expression keys - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet ds2 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet coGroupDs = - ds.coGroup(ds2).where("myInt").equalTo("myInt").with(new CustomTypeCoGroup()); - - List result = coGroupDs.collect(); - - String expected = - "1,0,test\n" - + "2,6,test\n" - + "3,24,test\n" - + "4,60,test\n" - + "5,120,test\n" - + "6,210,test\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testCoGroupOnTwoCustomTypeInputsWithExpressionKeyAndFieldSelector() - throws Exception { - /* - * CoGroup on two custom type inputs using expression keys - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet coGroupDs = - ds.coGroup(ds2).where("nestedPojo.longNumber").equalTo(6).with(new CoGroup1()); - - List result = coGroupDs.collect(); - - String expected = "-1,20000,Flink\n" + "-1,10000,Flink\n" + "-1,30000,Flink\n"; - - compareResultAsText(result, expected); - } - - private static class CoGroup1 - implements CoGroupFunction< - POJO, - Tuple7, - CustomType> { - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable first, - Iterable> second, - Collector out) - throws Exception { - for (POJO p : first) { - for (Tuple7 t : second) { - Assert.assertTrue(p.nestedPojo.longNumber == t.f6); - out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink")); - } - } - } - } - - @Test - public void testCoGroupFieldSelectorAndComplicatedKeySelector() throws Exception { - /* - * CoGroup field-selector (expression keys) + key selector function - * The key selector is unnecessary complicated (Tuple1) ;) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet coGroupDs = - ds.coGroup(ds2).where(new KeySelector6()).equalTo(6).with(new CoGroup3()); - - List result = coGroupDs.collect(); - - String expected = "-1,20000,Flink\n" + "-1,10000,Flink\n" + "-1,30000,Flink\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector6 implements KeySelector> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple1 getKey(POJO value) throws Exception { - return new Tuple1(value.nestedPojo.longNumber); - } - } - - private static class CoGroup3 - implements CoGroupFunction< - POJO, - Tuple7, - CustomType> { - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable first, - Iterable> second, - Collector out) - throws Exception { - for (POJO p : first) { - for (Tuple7 t : second) { - Assert.assertTrue(p.nestedPojo.longNumber == t.f6); - out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink")); - } - } - } - } - - @Test - public void testCoGroupFieldSelectorAndKeySelector() throws Exception { - /* - * CoGroup field-selector (expression keys) + key selector function - * The key selector is simple here - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet coGroupDs = - ds.coGroup(ds2).where(new KeySelector1()).equalTo(6).with(new CoGroup2()); - - List result = coGroupDs.collect(); - - String expected = "-1,20000,Flink\n" + "-1,10000,Flink\n" + "-1,30000,Flink\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testCoGroupWithAtomicType1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet ds2 = env.fromElements(0, 1, 2); - - DataSet> coGroupDs = - ds1.coGroup(ds2).where(0).equalTo("*").with(new CoGroupAtomic1()); - - List> result = coGroupDs.collect(); - - String expected = "(1,1,Hi)\n" + "(2,2,Hello)"; - - compareResultAsText(result, expected); - } - - @Test - public void testCoGroupWithAtomicType2() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds1 = env.fromElements(0, 1, 2); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - - DataSet> coGroupDs = - ds1.coGroup(ds2).where("*").equalTo(0).with(new CoGroupAtomic2()); - - List> result = coGroupDs.collect(); - - String expected = "(1,1,Hi)\n" + "(2,2,Hello)"; - - compareResultAsText(result, expected); - } - - @Test - public void testCoGroupWithRangePartitioning() throws Exception { - /* - * Test coGroup on tuples with multiple key field positions and same customized distribution - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - - env.setParallelism(4); - TestDistribution testDis = new TestDistribution(); - DataSet> coGrouped = - DataSetUtils.partitionByRange(ds1, testDis, 0, 4) - .coGroup(DataSetUtils.partitionByRange(ds2, testDis, 0, 1)) - .where(0, 4) - .equalTo(0, 1) - .with(new Tuple5Tuple3CoGroup()); - - List> result = coGrouped.collect(); - - String expected = - "1,1,Hallo\n" - + "2,2,Hallo Welt\n" - + "3,2,Hallo Welt wie gehts?\n" - + "3,2,ABC\n" - + "5,3,HIJ\n" - + "5,3,IJK\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCoGroupLambda() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> left = - env.fromElements( - new Tuple2<>(1, "hello"), new Tuple2<>(2, "what's"), new Tuple2<>(2, "up")); - DataSet> right = - env.fromElements( - new Tuple2<>(1, "not"), new Tuple2<>(1, "much"), new Tuple2<>(2, "really")); - DataSet joined = - left.coGroup(right) - .where(0) - .equalTo(0) - .with( - (Iterable> values1, - Iterable> values2, - Collector out) -> { - int sum = 0; - for (Tuple2 next : values1) { - sum += next.f0; - } - for (Tuple2 next : values2) { - sum += next.f0; - } - out.collect(sum); - }) - .returns(Integer.class); - List result = joined.collect(); - - String expected = "6\n3\n"; - - compareResultAsText(result, expected); - } - - // -------------------------------------------------------------------------------------------- - // UDF classes - // -------------------------------------------------------------------------------------------- - - private static class KeySelector1 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Long getKey(POJO value) throws Exception { - return value.nestedPojo.longNumber; - } - } - - private static class CoGroup2 - implements CoGroupFunction< - POJO, - Tuple7, - CustomType> { - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable first, - Iterable> second, - Collector out) - throws Exception { - for (POJO p : first) { - for (Tuple7 t : second) { - Assert.assertTrue(p.nestedPojo.longNumber == t.f6); - out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink")); - } - } - } - } - - private static class Tuple5CoGroup - implements CoGroupFunction< - Tuple5, - Tuple5, - Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) { - int sum = 0; - int id = 0; - - for (Tuple5 element : first) { - sum += element.f2; - id = element.f0; - } - - for (Tuple5 element : second) { - sum += element.f2; - id = element.f0; - } - - out.collect(new Tuple2(id, sum)); - } - } - - private static class CustomTypeCoGroup - implements CoGroupFunction { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable first, - Iterable second, - Collector out) { - - CustomType o = new CustomType(0, 0, "test"); - - for (CustomType element : first) { - o.myInt = element.myInt; - o.myLong += element.myLong; - } - - for (CustomType element : second) { - o.myInt = element.myInt; - o.myLong += element.myLong; - } - - out.collect(o); - } - } - - private static class MixedCoGroup - implements CoGroupFunction< - Tuple5, - CustomType, - Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable second, - Collector> out) - throws Exception { - - long sum = 0; - int id = 0; - - for (Tuple5 element : first) { - sum += element.f0; - id = element.f2; - } - - for (CustomType element : second) { - id = element.myInt; - sum += element.myLong; - } - - out.collect(new Tuple3(id, sum, "test")); - } - } - - private static class MixedCoGroup2 - implements CoGroupFunction< - CustomType, Tuple5, CustomType> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable first, - Iterable> second, - Collector out) { - CustomType o = new CustomType(0, 0, "test"); - - for (CustomType element : first) { - o.myInt = element.myInt; - o.myLong += element.myLong; - } - - for (Tuple5 element : second) { - o.myInt = element.f2; - o.myLong += element.f0; - } - - out.collect(o); - } - } - - private static class Tuple3ReturnLeft - implements CoGroupFunction< - Tuple3, - Tuple3, - Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) { - for (Tuple3 element : first) { - if (element.f0 < 6) { - out.collect(element); - } - } - } - } - - private static class Tuple5ReturnRight - implements CoGroupFunction< - Tuple5, - Tuple5, - Tuple5> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) { - for (Tuple5 element : second) { - if (element.f0 < 4) { - out.collect(element); - } - } - } - } - - private static class Tuple5CoGroupBC - extends RichCoGroupFunction< - Tuple5, - Tuple5, - Tuple3> { - - private static final long serialVersionUID = 1L; - - private int broadcast = 42; - - @Override - public void open(OpenContext openContext) { - - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - broadcast = sum; - } - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) { - int sum = 0; - int id = 0; - - for (Tuple5 element : first) { - sum += element.f2; - id = element.f0; - } - - for (Tuple5 element : second) { - sum += element.f2; - id = element.f0; - } - - out.collect(new Tuple3(id, sum, broadcast)); - } - } - - private static class Tuple5Tuple3CoGroup - implements CoGroupFunction< - Tuple5, - Tuple3, - Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) { - List strs = new ArrayList(); - - for (Tuple5 t : first) { - strs.add(t.f3); - } - - for (Tuple3 t : second) { - for (String s : strs) { - out.collect(new Tuple3(t.f0, t.f1, s)); - } - } - } - } - - private static class CoGroupAtomic1 - implements CoGroupFunction< - Tuple3, Integer, Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable> first, - Iterable second, - Collector> out) - throws Exception { - List ints = new ArrayList(); - - for (Integer i : second) { - ints.add(i); - } - - for (Tuple3 t : first) { - for (Integer i : ints) { - if (t.f0.equals(i)) { - out.collect(t); - } - } - } - } - } - - private static class CoGroupAtomic2 - implements CoGroupFunction< - Integer, Tuple3, Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public void coGroup( - Iterable first, - Iterable> second, - Collector> out) - throws Exception { - List ints = new ArrayList(); - - for (Integer i : first) { - ints.add(i); - } - - for (Tuple3 t : second) { - for (Integer i : ints) { - if (t.f0.equals(i)) { - out.collect(t); - } - } - } - } - } - - /** Test {@link DataDistribution}. */ - public static class TestDistribution implements DataDistribution { - public Object[][] boundaries = - new Object[][] { - new Object[] {2, 2L}, - new Object[] {5, 4L}, - new Object[] {10, 12L}, - new Object[] {21, 6L} - }; - - public TestDistribution() {} - - @Override - public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - return boundaries[bucketNum]; - } - - @Override - public int getNumberOfFields() { - return 2; - } - - @Override - public TypeInformation[] getKeyTypes() { - return new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO - }; - } - - @Override - public void write(DataOutputView out) throws IOException {} - - @Override - public void read(DataInputView in) throws IOException {} - - @Override - public boolean equals(Object obj) { - return obj instanceof TestDistribution; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java deleted file mode 100644 index 9782541ddf795..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java +++ /dev/null @@ -1,508 +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.operators; - -import org.apache.flink.api.common.functions.CrossFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichCrossFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple6; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link CrossFunction} and {@link RichCrossFunction}. */ -@RunWith(Parameterized.class) -public class CrossITCase extends MultipleProgramsTestBaseJUnit4 { - - public CrossITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testCorretnessOfCrossOnTwoTupleInputs() throws Exception { - /* - * check correctness of cross on two tuple inputs - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = ds.cross(ds2).with(new Tuple5Cross()); - - List> result = crossDs.collect(); - - String expected = - "0,HalloHallo\n" - + "1,HalloHallo Welt\n" - + "2,HalloHallo Welt wie\n" - + "1,Hallo WeltHallo\n" - + "2,Hallo WeltHallo Welt\n" - + "3,Hallo WeltHallo Welt wie\n" - + "2,Hallo Welt wieHallo\n" - + "3,Hallo Welt wieHallo Welt\n" - + "4,Hallo Welt wieHallo Welt wie\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCrossIfUDFReturnsLeftInputObject() throws Exception { - /* - * check correctness of cross if UDF returns left input object - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = ds.cross(ds2).with(new Tuple3ReturnLeft()); - - List> result = crossDs.collect(); - - String expected = - "1,1,Hi\n" - + "1,1,Hi\n" - + "1,1,Hi\n" - + "2,2,Hello\n" - + "2,2,Hello\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "3,2,Hello world\n" - + "3,2,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCrossIfUDFReturnsRightInputObject() throws Exception { - /* - * check correctness of cross if UDF returns right input object - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = - ds.cross(ds2).with(new Tuple5ReturnRight()); - - List> result = crossDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "1,1,0,Hallo,1\n" - + "1,1,0,Hallo,1\n" - + "2,2,1,Hallo Welt,2\n" - + "2,2,1,Hallo Welt,2\n" - + "2,2,1,Hallo Welt,2\n" - + "2,3,2,Hallo Welt wie,1\n" - + "2,3,2,Hallo Welt wie,1\n" - + "2,3,2,Hallo Welt wie,1\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCrossWithBroadcastSet() throws Exception { - /* - * check correctness of cross with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = - ds.cross(ds2).with(new Tuple5CrossBC()).withBroadcastSet(intDs, "ints"); - - List> result = crossDs.collect(); - - String expected = - "2,0,55\n" - + "3,0,55\n" - + "3,0,55\n" - + "3,0,55\n" - + "4,1,55\n" - + "4,2,55\n" - + "3,0,55\n" - + "4,2,55\n" - + "4,4,55\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCrossWithHuge() throws Exception { - /* - * check correctness of crossWithHuge (only correctness of result -> should be the same as with normal cross) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = ds.crossWithHuge(ds2).with(new Tuple5Cross()); - - List> result = crossDs.collect(); - - String expected = - "0,HalloHallo\n" - + "1,HalloHallo Welt\n" - + "2,HalloHallo Welt wie\n" - + "1,Hallo WeltHallo\n" - + "2,Hallo WeltHallo Welt\n" - + "3,Hallo WeltHallo Welt wie\n" - + "2,Hallo Welt wieHallo\n" - + "3,Hallo Welt wieHallo Welt\n" - + "4,Hallo Welt wieHallo Welt wie\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCrossWithTiny() throws Exception { - /* - * check correctness of crossWithTiny (only correctness of result -> should be the same as with normal cross) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = ds.crossWithTiny(ds2).with(new Tuple5Cross()); - - List> result = crossDs.collect(); - - String expected = - "0,HalloHallo\n" - + "1,HalloHallo Welt\n" - + "2,HalloHallo Welt wie\n" - + "1,Hallo WeltHallo\n" - + "2,Hallo WeltHallo Welt\n" - + "3,Hallo WeltHallo Welt wie\n" - + "2,Hallo Welt wieHallo\n" - + "3,Hallo Welt wieHallo Welt\n" - + "4,Hallo Welt wieHallo Welt wie\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testProjectCrossOnATupleInput1() throws Exception { - /* - * project cross on a tuple input 1 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = - ds.cross(ds2) - .projectFirst(2, 1) - .projectSecond(3) - .projectFirst(0) - .projectSecond(4, 1); - - List> result = crossDs.collect(); - - String expected = - "Hi,1,Hallo,1,1,1\n" - + "Hi,1,Hallo Welt,1,2,2\n" - + "Hi,1,Hallo Welt wie,1,1,3\n" - + "Hello,2,Hallo,2,1,1\n" - + "Hello,2,Hallo Welt,2,2,2\n" - + "Hello,2,Hallo Welt wie,2,1,3\n" - + "Hello world,2,Hallo,3,1,1\n" - + "Hello world,2,Hallo Welt,3,2,2\n" - + "Hello world,2,Hallo Welt wie,3,1,3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testProjectCrossOnATupleInput2() throws Exception { - /* - * project cross on a tuple input 2 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> crossDs = - ds.cross(ds2) - .projectSecond(3) - .projectFirst(2, 1) - .projectSecond(4, 1) - .projectFirst(0); - - List> result = crossDs.collect(); - - String expected = - "Hallo,Hi,1,1,1,1\n" - + "Hallo Welt,Hi,1,2,2,1\n" - + "Hallo Welt wie,Hi,1,1,3,1\n" - + "Hallo,Hello,2,1,1,2\n" - + "Hallo Welt,Hello,2,2,2,2\n" - + "Hallo Welt wie,Hello,2,1,3,2\n" - + "Hallo,Hello world,2,1,1,3\n" - + "Hallo Welt,Hello world,2,2,2,3\n" - + "Hallo Welt wie,Hello world,2,1,3,3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfDefaultCross() throws Exception { - /* - * check correctness of default cross - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet, Tuple5>> - crossDs = ds.cross(ds2); - - List, Tuple5>> - result = crossDs.collect(); - - String expected = - "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" - + "(1,1,Hi),(1,1,0,Hallo,1)\n" - + "(1,1,Hi),(2,3,2,Hallo Welt wie,1)\n" - + "(2,2,Hello),(2,2,1,Hallo Welt,2)\n" - + "(2,2,Hello),(1,1,0,Hallo,1)\n" - + "(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" - + "(3,2,Hello world),(2,2,1,Hallo Welt,2)\n" - + "(3,2,Hello world),(1,1,0,Hallo,1)\n" - + "(3,2,Hello world),(2,3,2,Hallo Welt wie,1)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfCrossOnTwoCustomTypeInputs() throws Exception { - /* - * check correctness of cross on two custom type inputs - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet crossDs = ds.cross(ds2).with(new CustomTypeCross()); - - List result = crossDs.collect(); - - String expected = - "1,0,HiHi\n" - + "2,1,HiHello\n" - + "2,2,HiHello world\n" - + "2,1,HelloHi\n" - + "4,2,HelloHello\n" - + "4,3,HelloHello world\n" - + "2,2,Hello worldHi\n" - + "4,3,Hello worldHello\n" - + "4,4,Hello worldHello world"; - - compareResultAsText(result, expected); - } - - @Test - public void testCorrectnessOfCrossATupleInputAndACustomTypeInput() throws Exception { - /* - * check correctness of cross a tuple input and a custom type input - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet> crossDs = ds.cross(ds2).with(new MixedCross()); - - List> result = crossDs.collect(); - - String expected = - "2,0,HalloHi\n" - + "3,0,HalloHello\n" - + "3,0,HalloHello world\n" - + "3,0,Hallo WeltHi\n" - + "4,1,Hallo WeltHello\n" - + "4,2,Hallo WeltHello world\n" - + "3,0,Hallo Welt wieHi\n" - + "4,2,Hallo Welt wieHello\n" - + "4,4,Hallo Welt wieHello world\n"; - - compareResultAsTuples(result, expected); - } - - private static class Tuple5Cross - implements CrossFunction< - Tuple5, - Tuple5, - Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 cross( - Tuple5 first, - Tuple5 second) - throws Exception { - - return new Tuple2(first.f2 + second.f2, first.f3 + second.f3); - } - } - - private static class CustomTypeCross - implements CrossFunction { - - private static final long serialVersionUID = 1L; - - @Override - public CustomType cross(CustomType first, CustomType second) throws Exception { - - return new CustomType( - first.myInt * second.myInt, - first.myLong + second.myLong, - first.myString + second.myString); - } - } - - private static class MixedCross - implements CrossFunction< - Tuple5, - CustomType, - Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 cross( - Tuple5 first, CustomType second) - throws Exception { - - return new Tuple3( - first.f0 + second.myInt, first.f2 * second.myLong, first.f3 + second.myString); - } - } - - private static class Tuple3ReturnLeft - implements CrossFunction< - Tuple3, - Tuple5, - Tuple3> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 cross( - Tuple3 first, - Tuple5 second) - throws Exception { - - return first; - } - } - - private static class Tuple5ReturnRight - implements CrossFunction< - Tuple3, - Tuple5, - Tuple5> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple5 cross( - Tuple3 first, - Tuple5 second) - throws Exception { - - return second; - } - } - - private static class Tuple5CrossBC - extends RichCrossFunction< - Tuple5, - Tuple5, - Tuple3> { - - private static final long serialVersionUID = 1L; - - private int broadcast = 42; - - @Override - public void open(OpenContext openContext) { - - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - broadcast = sum; - } - - @Override - public Tuple3 cross( - Tuple5 first, - Tuple5 second) - throws Exception { - - return new Tuple3( - first.f0 + second.f0, first.f2 * second.f2, broadcast); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java deleted file mode 100644 index eb165774782f3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java +++ /dev/null @@ -1,364 +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.operators; - -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.RichMapPartitionFunction; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.Tuple3; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.Collector; -import org.apache.flink.util.TestLogger; - -import org.junit.ClassRule; -import org.junit.Test; - -import java.io.IOException; - -import static org.junit.Assert.fail; - -/** Integration tests for custom {@link DataDistribution}. */ -@SuppressWarnings("serial") -public class CustomDistributionITCase extends TestLogger { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(8) - .build()); - - // ------------------------------------------------------------------------ - - /** - * Test the record partitioned rightly with one field according to the customized data - * distribution. - */ - @Test - public void testPartitionWithDistribution1() throws Exception { - final TestDataDist1 dist = new TestDataDist1(); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(dist.getParallelism()); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - - DataSet result = - DataSetUtils.partitionByRange(input, dist, 0) - .mapPartition( - new RichMapPartitionFunction< - Tuple3, Boolean>() { - - @Override - public void mapPartition( - Iterable> values, - Collector out) - throws Exception { - int pIdx = - getRuntimeContext() - .getTaskInfo() - .getIndexOfThisSubtask(); - - for (Tuple3 s : values) { - boolean correctlyPartitioned = true; - if (pIdx == 0) { - Integer[] upper = dist.boundaries[0]; - if (s.f0.compareTo(upper[0]) > 0) { - correctlyPartitioned = false; - } - } else if (pIdx > 0 - && pIdx < dist.getParallelism() - 1) { - Integer[] lower = dist.boundaries[pIdx - 1]; - Integer[] upper = dist.boundaries[pIdx]; - if (s.f0.compareTo(upper[0]) > 0 - || (s.f0.compareTo(lower[0]) <= 0)) { - correctlyPartitioned = false; - } - } else { - Integer[] lower = dist.boundaries[pIdx - 1]; - if ((s.f0.compareTo(lower[0]) <= 0)) { - correctlyPartitioned = false; - } - } - - if (!correctlyPartitioned) { - fail( - "Record was not correctly partitioned: " - + s.toString()); - } - } - } - }); - - result.output(new DiscardingOutputFormat()); - env.execute(); - } - - /** - * Test the record partitioned rightly with two fields according to the customized data - * distribution. - */ - @Test - public void testRangeWithDistribution2() throws Exception { - final TestDataDist2 dist = new TestDataDist2(); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(dist.getParallelism()); - - DataSet> input = - env.fromElements( - new Tuple3<>(1, 5, "Hi"), - new Tuple3<>(1, 6, "Hi"), - new Tuple3<>(1, 7, "Hi"), - new Tuple3<>(1, 11, "Hello"), - new Tuple3<>(2, 3, "World"), - new Tuple3<>(2, 4, "World"), - new Tuple3<>(2, 5, "World"), - new Tuple3<>(2, 13, "Hello World"), - new Tuple3<>(3, 8, "Say"), - new Tuple3<>(4, 0, "Why"), - new Tuple3<>(4, 2, "Java"), - new Tuple3<>(4, 11, "Say Hello"), - new Tuple3<>(5, 1, "Hi Java!"), - new Tuple3<>(5, 2, "Hi Java?"), - new Tuple3<>(5, 3, "Hi Java again")); - - DataSet result = - DataSetUtils.partitionByRange(input, dist, 0, 1) - .mapPartition( - new RichMapPartitionFunction< - Tuple3, Boolean>() { - - @Override - public void mapPartition( - Iterable> values, - Collector out) - throws Exception { - int pIdx = - getRuntimeContext() - .getTaskInfo() - .getIndexOfThisSubtask(); - boolean correctlyPartitioned = true; - - for (Tuple3 s : values) { - - if (pIdx == 0) { - Integer[] upper = dist.boundaries[0]; - if (s.f0.compareTo(upper[0]) > 0 - || (s.f0.compareTo(upper[0]) == 0 - && s.f1.compareTo(upper[1]) > 0)) { - correctlyPartitioned = false; - } - } else if (pIdx > 0 - && pIdx < dist.getParallelism() - 1) { - Integer[] lower = dist.boundaries[pIdx - 1]; - Integer[] upper = dist.boundaries[pIdx]; - - if (s.f0.compareTo(upper[0]) > 0 - || (s.f0.compareTo(upper[0]) == 0 - && s.f1.compareTo(upper[1]) > 0) - || (s.f0.compareTo(lower[0]) < 0) - || (s.f0.compareTo(lower[0]) == 0 - && s.f1.compareTo(lower[1]) <= 0)) { - correctlyPartitioned = false; - } - } else { - Integer[] lower = dist.boundaries[pIdx - 1]; - if ((s.f0.compareTo(lower[0]) < 0) - || (s.f0.compareTo(lower[0]) == 0 - && s.f1.compareTo(lower[1]) <= 0)) { - correctlyPartitioned = false; - } - } - - if (!correctlyPartitioned) { - fail( - "Record was not correctly partitioned: " - + s.toString()); - } - } - } - }); - - result.output(new DiscardingOutputFormat()); - env.execute(); - } - - /* - * Test the number of partition keys less than the number of distribution fields - */ - @Test - public void testPartitionKeyLessDistribution() throws Exception { - final TestDataDist2 dist = new TestDataDist2(); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(dist.getParallelism()); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - - DataSet result = - DataSetUtils.partitionByRange(input, dist, 0) - .mapPartition( - new RichMapPartitionFunction< - Tuple3, Boolean>() { - - @Override - public void mapPartition( - Iterable> values, - Collector out) - throws Exception { - int pIdx = - getRuntimeContext() - .getTaskInfo() - .getIndexOfThisSubtask(); - - for (Tuple3 s : values) { - boolean correctlyPartitioned = true; - if (pIdx == 0) { - Integer[] upper = dist.boundaries[0]; - if (s.f0.compareTo(upper[0]) > 0) { - correctlyPartitioned = false; - } - } else if (pIdx > 0 - && pIdx < dist.getParallelism() - 1) { - Integer[] lower = dist.boundaries[pIdx - 1]; - Integer[] upper = dist.boundaries[pIdx]; - if (s.f0.compareTo(upper[0]) > 0 - || (s.f0.compareTo(lower[0]) <= 0)) { - correctlyPartitioned = false; - } - } else { - Integer[] lower = dist.boundaries[pIdx - 1]; - if ((s.f0.compareTo(lower[0]) <= 0)) { - correctlyPartitioned = false; - } - } - - if (!correctlyPartitioned) { - fail( - "Record was not correctly partitioned: " - + s.toString()); - } - } - } - }); - - result.output(new DiscardingOutputFormat()); - env.execute(); - } - - /* - * Test the number of partition keys larger than the number of distribution fields - */ - @Test(expected = IllegalArgumentException.class) - public void testPartitionMoreThanDistribution() throws Exception { - final TestDataDist2 dist = new TestDataDist2(); - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - DataSetUtils.partitionByRange(input, dist, 0, 1, 2); - } - - /** The class is used to do the tests of range partition with one key. */ - public static class TestDataDist1 implements DataDistribution { - - public Integer[][] boundaries = - new Integer[][] { - new Integer[] {4}, new Integer[] {9}, new Integer[] {13}, new Integer[] {18} - }; - - public TestDataDist1() {} - - public int getParallelism() { - return boundaries.length; - } - - @Override - public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - return boundaries[bucketNum]; - } - - @Override - public int getNumberOfFields() { - return 1; - } - - @Override - public TypeInformation[] getKeyTypes() { - return new TypeInformation[] {BasicTypeInfo.INT_TYPE_INFO}; - } - - @Override - public void write(DataOutputView out) throws IOException {} - - @Override - public void read(DataInputView in) throws IOException {} - } - - /** The class is used to do the tests of range partition with two keys. */ - public static class TestDataDist2 implements DataDistribution { - - public Integer[][] boundaries = - new Integer[][] { - new Integer[] {1, 6}, - new Integer[] {2, 4}, - new Integer[] {3, 9}, - new Integer[] {4, 1}, - new Integer[] {5, 2} - }; - - public TestDataDist2() {} - - public int getParallelism() { - return boundaries.length; - } - - @Override - public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - return boundaries[bucketNum]; - } - - @Override - public int getNumberOfFields() { - return 2; - } - - @Override - public TypeInformation[] getKeyTypes() { - return new TypeInformation[] {BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO}; - } - - @Override - public void write(DataOutputView out) throws IOException {} - - @Override - public void read(DataInputView in) throws IOException {} - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java deleted file mode 100644 index 3b44574217de3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java +++ /dev/null @@ -1,365 +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.operators; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.Order; -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.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.BufferedReader; -import java.io.File; -import java.util.Random; -import java.util.UUID; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemoryWithStrictOrder; -import static org.apache.flink.test.util.TestBaseUtils.getResultReader; -import static org.junit.Assert.assertTrue; - -/** Tests for data sinks. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class DataSinkITCase extends MultipleProgramsTestBaseJUnit4 { - - public DataSinkITCase(TestExecutionMode mode) { - super(mode); - } - - private String resultPath; - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Before - public void before() throws Exception { - final File folder = tempFolder.newFolder(); - final File resultFile = new File(folder, UUID.randomUUID().toString()); - resultPath = resultFile.toURI().toString(); - } - - @Test - public void testIntSortingParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getIntegerDataSet(env); - ds.writeAsText(resultPath).sortLocalOutput("*", Order.DESCENDING).setParallelism(1); - - env.execute(); - - String expected = "5\n5\n5\n5\n5\n4\n4\n4\n4\n3\n3\n3\n2\n2\n1\n"; - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testStringSortingParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getStringDataSet(env); - ds.writeAsText(resultPath).sortLocalOutput("*", Order.ASCENDING).setParallelism(1); - - env.execute(); - - String expected = - "Hello\n" - + "Hello world\n" - + "Hello world, how are you?\n" - + "Hi\n" - + "I am fine.\n" - + "LOL\n" - + "Luke Skywalker\n" - + "Random comment\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testTupleSortingSingleAscParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - ds.writeAsCsv(resultPath).sortLocalOutput(0, Order.ASCENDING).setParallelism(1); - - env.execute(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "6,3,Luke Skywalker\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "9,4,Comment#3\n" - + "10,4,Comment#4\n" - + "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n" - + "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "18,6,Comment#12\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "21,6,Comment#15\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testTupleSortingSingleDescParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - ds.writeAsCsv(resultPath).sortLocalOutput(0, Order.DESCENDING).setParallelism(1); - - env.execute(); - - String expected = - "21,6,Comment#15\n" - + "20,6,Comment#14\n" - + "19,6,Comment#13\n" - + "18,6,Comment#12\n" - + "17,6,Comment#11\n" - + "16,6,Comment#10\n" - + "15,5,Comment#9\n" - + "14,5,Comment#8\n" - + "13,5,Comment#7\n" - + "12,5,Comment#6\n" - + "11,5,Comment#5\n" - + "10,4,Comment#4\n" - + "9,4,Comment#3\n" - + "8,4,Comment#2\n" - + "7,4,Comment#1\n" - + "6,3,Luke Skywalker\n" - + "5,3,I am fine.\n" - + "4,3,Hello world, how are you?\n" - + "3,2,Hello world\n" - + "2,2,Hello\n" - + "1,1,Hi\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testTupleSortingDualParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - ds.writeAsCsv(resultPath) - .sortLocalOutput(1, Order.DESCENDING) - .sortLocalOutput(0, Order.ASCENDING) - .setParallelism(1); - - env.execute(); - - String expected = - "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "18,6,Comment#12\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "21,6,Comment#15\n" - + "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "9,4,Comment#3\n" - + "10,4,Comment#4\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "6,3,Luke Skywalker\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "1,1,Hi\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testTupleSortingNestedParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet, String, Integer>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet2(env); - ds.writeAsText(resultPath) - .sortLocalOutput("f0.f1", Order.ASCENDING) - .sortLocalOutput("f1", Order.DESCENDING) - .setParallelism(1); - - env.execute(); - - String expected = - "((2,1),a,3)\n" - + "((2,2),b,4)\n" - + "((1,2),a,1)\n" - + "((3,3),c,5)\n" - + "((1,3),a,2)\n" - + "((3,6),c,6)\n" - + "((4,9),c,7)\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testTupleSortingNestedParallelism1_2() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet, String, Integer>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet2(env); - ds.writeAsText(resultPath) - .sortLocalOutput(1, Order.ASCENDING) - .sortLocalOutput(2, Order.DESCENDING) - .setParallelism(1); - - env.execute(); - - String expected = - "((2,1),a,3)\n" - + "((1,3),a,2)\n" - + "((1,2),a,1)\n" - + "((2,2),b,4)\n" - + "((4,9),c,7)\n" - + "((3,6),c,6)\n" - + "((3,3),c,5)\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testPojoSortingSingleParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); - ds.writeAsText(resultPath).sortLocalOutput("number", Order.ASCENDING).setParallelism(1); - - env.execute(); - - String expected = - "1 First (10,100,1000,One) 10100\n" - + "2 First_ (10,105,1000,One) 10200\n" - + "3 First (11,102,3000,One) 10200\n" - + "4 First_ (11,106,1000,One) 10300\n" - + "5 First (11,102,2000,One) 10100\n" - + "6 Second_ (20,200,2000,Two) 10100\n" - + "7 Third (31,301,2000,Three) 10200\n" - + "8 Third_ (30,300,1000,Three) 10100\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testPojoSortingDualParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); - ds.writeAsText(resultPath) - .sortLocalOutput("str", Order.ASCENDING) - .sortLocalOutput("number", Order.DESCENDING) - .setParallelism(1); - - env.execute(); - - String expected = - "5 First (11,102,2000,One) 10100\n" - + "3 First (11,102,3000,One) 10200\n" - + "1 First (10,100,1000,One) 10100\n" - + "4 First_ (11,106,1000,One) 10300\n" - + "2 First_ (10,105,1000,One) 10200\n" - + "6 Second_ (20,200,2000,Two) 10100\n" - + "7 Third (31,301,2000,Three) 10200\n" - + "8 Third_ (30,300,1000,Three) 10100\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testPojoSortingNestedParallelism1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); - ds.writeAsText(resultPath) - .sortLocalOutput("nestedTupleWithCustom.f0", Order.ASCENDING) - .sortLocalOutput("nestedTupleWithCustom.f1.myInt", Order.DESCENDING) - .sortLocalOutput("nestedPojo.longNumber", Order.ASCENDING) - .setParallelism(1); - - env.execute(); - - String expected = - "2 First_ (10,105,1000,One) 10200\n" - + "1 First (10,100,1000,One) 10100\n" - + "4 First_ (11,106,1000,One) 10300\n" - + "5 First (11,102,2000,One) 10100\n" - + "3 First (11,102,3000,One) 10200\n" - + "6 Second_ (20,200,2000,Two) 10100\n" - + "8 Third_ (30,300,1000,Three) 10100\n" - + "7 Third (31,301,2000,Three) 10200\n"; - - compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath); - } - - @Test - public void testSortingParallelism4() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = env.generateSequence(0, 1000); - // randomize - ds.map( - new MapFunction() { - - Random rand = new Random(1234L); - - @Override - public Long map(Long value) throws Exception { - return rand.nextLong(); - } - }) - .writeAsText(resultPath) - .sortLocalOutput("*", Order.ASCENDING) - .setParallelism(4); - - env.execute(); - - BufferedReader[] resReaders = getResultReader(resultPath); - for (BufferedReader br : resReaders) { - long cmp = Long.MIN_VALUE; - while (br.ready()) { - long cur = Long.parseLong(br.readLine()); - assertTrue("Invalid order of sorted output", cmp <= cur); - cmp = cur; - } - br.close(); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java deleted file mode 100644 index abf637bd105fc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java +++ /dev/null @@ -1,78 +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.operators; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import org.junit.Assert; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; - -/** Tests for the DataSource. */ -public class DataSourceITCase extends JavaProgramTestBaseJUnit4 { - - private String inputPath; - - @Override - protected void preSubmit() throws Exception { - inputPath = createTempFile("input", "ab\n" + "cd\n" + "ef\n"); - } - - @Override - protected void testProgram() throws Exception { - /* - * Test passing a configuration object to an input format - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - Configuration ifConf = new Configuration(); - ifConf.setString("prepend", "test"); - - DataSet ds = - env.createInput(new TestInputFormat(new Path(inputPath))).withParameters(ifConf); - List result = ds.collect(); - - String expectedResult = "ab\n" + "cd\n" + "ef\n"; - - compareResultAsText(result, expectedResult); - } - - private static class TestInputFormat extends TextInputFormat { - private static final long serialVersionUID = 1L; - - public TestInputFormat(Path filePath) { - super(filePath); - } - - @Override - public void configure(Configuration parameters) { - super.configure(parameters); - - Assert.assertNotNull(parameters.getString("prepend", null)); - Assert.assertEquals("test", parameters.getString("prepend", null)); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java deleted file mode 100644 index eaa3cb5e51276..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java +++ /dev/null @@ -1,314 +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.operators; - -import org.apache.flink.api.common.functions.MapFunction; -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.KeySelector; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link DataSet#distinct}. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class DistinctITCase extends MultipleProgramsTestBaseJUnit4 { - - public DistinctITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelector() throws Exception { - /* - * check correctness of distinct on tuples with key field selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> distinctDs = ds.union(ds).distinct(0, 1, 2); - - List> result = distinctDs.collect(); - - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelectorWithNotAllFieldsSelected() - throws Exception { - /* - * check correctness of distinct on tuples with key field selector with not all fields selected - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> distinctDs = ds.union(ds).distinct(0).project(0); - - List> result = distinctDs.collect(); - - String expected = "1\n" + "2\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfDistinctOnTuplesWithKeyExtractorFunction() throws Exception { - /* - * check correctness of distinct on tuples with key extractor function - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> reduceDs = ds.union(ds).distinct(new KeySelector1()).project(0); - - List> result = reduceDs.collect(); - - String expected = "1\n" + "2\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector1 - implements KeySelector, Integer> { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(Tuple5 in) { - return in.f0; - } - } - - @Test - public void testCorrectnessOfDistinctOnCustomTypeWithTypeExtractor() throws Exception { - /* - * check correctness of distinct on custom type with type extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet> reduceDs = ds.distinct(new KeySelector3()).map(new Mapper3()); - - List> result = reduceDs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsTuples(result, expected); - } - - private static class Mapper3 extends RichMapFunction> { - @Override - public Tuple1 map(CustomType value) throws Exception { - return new Tuple1(value.myInt); - } - } - - private static class KeySelector3 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testCorrectnessOfDistinctOnTuples() throws Exception { - /* - * check correctness of distinct on tuples - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> distinctDs = ds.union(ds).distinct(); - - List> result = distinctDs.collect(); - - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor() - throws Exception { - /* - * check correctness of distinct on custom type with tuple-returning type extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = ds.distinct(new KeySelector2()).project(0, 4); - - List> result = reduceDs.collect(); - - String expected = - "1,1\n" + "2,1\n" + "2,2\n" + "3,2\n" + "3,3\n" + "4,1\n" + "4,2\n" + "5,1\n" - + "5,2\n" + "5,3\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector2 - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 t) { - return new Tuple2(t.f0, t.f4); - } - } - - @Test - public void testCorrectnessOfDistinctOnTuplesWithFieldExpressions() throws Exception { - /* - * check correctness of distinct on tuples with field expressions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> reduceDs = ds.union(ds).distinct("f0").project(0); - - List> result = reduceDs.collect(); - - String expected = "1\n" + "2\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfDistinctOnPojos() throws Exception { - /* - * check correctness of distinct on Pojos - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); - DataSet reduceDs = ds.distinct("nestedPojo.longNumber").map(new Mapper2()); - - List result = reduceDs.collect(); - - String expected = "10000\n20000\n30000\n"; - - compareResultAsText(result, expected); - } - - private static class Mapper2 implements MapFunction { - @Override - public Integer map(POJO value) throws Exception { - return (int) value.nestedPojo.longNumber; - } - } - - @Test - public void testDistinctOnFullPojo() throws Exception { - /* - * distinct on full Pojo - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); - DataSet reduceDs = ds.distinct().map(new Mapper1()); - - List result = reduceDs.collect(); - - String expected = "10000\n20000\n30000\n"; - - compareResultAsText(result, expected); - } - - private static class Mapper1 implements MapFunction { - @Override - public Integer map(POJO value) throws Exception { - return (int) value.nestedPojo.longNumber; - } - } - - @Test - public void testCorrectnessOfDistinctOnAtomic() throws Exception { - /* - * check correctness of distinct on Integers - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds = CollectionDataSets.getIntegerDataSet(env); - DataSet reduceDs = ds.distinct(); - - List result = reduceDs.collect(); - - String expected = "1\n2\n3\n4\n5"; - - compareResultAsText(result, expected); - } - - @Test - public void testCorrectnessOfDistinctOnAtomicWithSelectAllChar() throws Exception { - /* - * check correctness of distinct on Strings, using Keys.ExpressionKeys.SELECT_ALL_CHAR - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds = CollectionDataSets.getStringDataSet(env); - DataSet reduceDs = ds.union(ds).distinct("*"); - - List result = reduceDs.collect(); - - String expected = - "I am fine.\n" - + "Luke Skywalker\n" - + "LOL\n" - + "Hello world, how are you?\n" - + "Hi\n" - + "Hello world\n" - + "Hello\n" - + "Random comment\n"; - - compareResultAsText(result, expected); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java deleted file mode 100644 index 964da3290b6c2..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java +++ /dev/null @@ -1,95 +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.operators; - -import org.apache.flink.api.common.functions.RichMapPartitionFunction; -import org.apache.flink.api.common.io.GenericInputFormat; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.util.Collector; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.io.IOException; -import java.util.List; - -import static org.junit.Assert.assertEquals; - -/** Test ExecutionEnvironment from user perspective. */ -@SuppressWarnings("serial") -public class ExecutionEnvironmentITCase extends TestLogger { - - private static final int PARALLELISM = 5; - - /** Ensure that the user can pass a custom configuration object to the LocalEnvironment. */ - @Test - public void testLocalEnvironmentWithConfig() throws Exception { - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.NUM_TASK_SLOTS, PARALLELISM); - - final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(conf); - - DataSet result = - env.createInput(new ParallelismDependentInputFormat()) - .rebalance() - .mapPartition( - new RichMapPartitionFunction() { - @Override - public void mapPartition( - Iterable values, Collector out) - throws Exception { - out.collect( - getRuntimeContext() - .getTaskInfo() - .getIndexOfThisSubtask()); - } - }); - List resultCollection = result.collect(); - assertEquals(PARALLELISM, resultCollection.size()); - } - - private static class ParallelismDependentInputFormat extends GenericInputFormat { - - private transient boolean emitted; - - @Override - public GenericInputSplit[] createInputSplits(int numSplits) throws IOException { - assertEquals(PARALLELISM, numSplits); - return super.createInputSplits(numSplits); - } - - @Override - public boolean reachedEnd() { - return emitted; - } - - @Override - public Integer nextRecord(Integer reuse) { - if (emitted) { - return null; - } - emitted = true; - return 1; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java deleted file mode 100644 index 2c532fc491868..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java +++ /dev/null @@ -1,322 +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.operators; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link FilterFunction} and {@link RichFilterFunction}. */ -@RunWith(Parameterized.class) -public class FilterITCase extends MultipleProgramsTestBaseJUnit4 { - public FilterITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testAllRejectingFilter() throws Exception { - /* - * Test all-rejecting filter. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = ds.filter(new Filter1()); - - List> result = filterDs.collect(); - - String expected = "\n"; - - compareResultAsTuples(result, expected); - } - - private static class Filter1 implements FilterFunction> { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Tuple3 value) throws Exception { - return false; - } - } - - @Test - public void testAllPassingFilter() throws Exception { - /* - * Test all-passing filter. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = ds.filter(new Filter2()); - List> result = filterDs.collect(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "6,3,Luke Skywalker\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "9,4,Comment#3\n" - + "10,4,Comment#4\n" - + "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n" - + "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "18,6,Comment#12\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "21,6,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class Filter2 implements FilterFunction> { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Tuple3 value) throws Exception { - return true; - } - } - - @Test - public void testFilterOnStringTupleField() throws Exception { - /* - * Test filter on String tuple field. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = ds.filter(new Filter3()); - List> result = filterDs.collect(); - - String expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"; - - compareResultAsTuples(result, expected); - } - - private static class Filter3 implements FilterFunction> { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Tuple3 value) throws Exception { - return value.f2.contains("world"); - } - } - - @Test - public void testFilterOnIntegerTupleField() throws Exception { - /* - * Test filter on Integer tuple field. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = ds.filter(new Filter4()); - List> result = filterDs.collect(); - - String expected = - "2,2,Hello\n" - + "4,3,Hello world, how are you?\n" - + "6,3,Luke Skywalker\n" - + "8,4,Comment#2\n" - + "10,4,Comment#4\n" - + "12,5,Comment#6\n" - + "14,5,Comment#8\n" - + "16,6,Comment#10\n" - + "18,6,Comment#12\n" - + "20,6,Comment#14\n"; - - compareResultAsTuples(result, expected); - } - - private static class Filter4 implements FilterFunction> { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Tuple3 value) throws Exception { - return (value.f0 % 2) == 0; - } - } - - @Test - public void testFilterBasicType() throws Exception { - /* - * Test filter on basic type - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getStringDataSet(env); - DataSet filterDs = ds.filter(new Filter5()); - List result = filterDs.collect(); - - String expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n"; - - compareResultAsText(result, expected); - } - - private static class Filter5 implements FilterFunction { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(String value) throws Exception { - return value.startsWith("H"); - } - } - - @Test - public void testFilterOnCustomType() throws Exception { - /* - * Test filter on custom type - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet filterDs = ds.filter(new Filter6()); - List result = filterDs.collect(); - - String expected = - "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n"; - - compareResultAsText(result, expected); - } - - private static class Filter6 implements FilterFunction { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(CustomType value) throws Exception { - return value.myString.contains("a"); - } - } - - @Test - public void testRichFilterOnStringTupleField() throws Exception { - /* - * Test filter on String tuple field. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ints = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = - ds.filter(new RichFilter1()).withBroadcastSet(ints, "ints"); - List> result = filterDs.collect(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n"; - - compareResultAsTuples(result, expected); - } - - private static class RichFilter1 extends RichFilterFunction> { - private static final long serialVersionUID = 1L; - - int literal = -1; - - @Override - public void open(OpenContext openContext) { - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - for (int i : ints) { - literal = literal < i ? i : literal; - } - } - - @Override - public boolean filter(Tuple3 value) throws Exception { - return value.f0 < literal; - } - } - - @Test - public void testFilterWithBroadcastVariables() throws Exception { - /* - * Test filter with broadcast variables - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> filterDs = - ds.filter(new RichFilter2()).withBroadcastSet(intDs, "ints"); - List> result = filterDs.collect(); - - String expected = - "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n"; - - compareResultAsTuples(result, expected); - } - - private static class RichFilter2 extends RichFilterFunction> { - private static final long serialVersionUID = 1L; - private int broadcastSum = 0; - - @Override - public void open(OpenContext openContext) { - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - for (Integer i : ints) { - broadcastSum += i; - } - } - - @Override - public boolean filter(Tuple3 value) throws Exception { - return (value.f1 == (broadcastSum / 11)); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java deleted file mode 100644 index dfab8bd071b01..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java +++ /dev/null @@ -1,163 +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.operators; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.Order; -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.GroupReduceOperator; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; - -/** Integration tests for {@link DataSet#first}. */ -@RunWith(Parameterized.class) -public class FirstNITCase extends MultipleProgramsTestBaseJUnit4 { - public FirstNITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testFirstNOnUngroupedDS() throws Exception { - /* - * First-n on ungrouped data set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> seven = ds.first(7).map(new OneMapper()).sum(0); - - List> result = seven.collect(); - - String expected = "(7)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testFirstNOnGroupedDS() throws Exception { - /* - * First-n on grouped data set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> first = - ds.groupBy(1).first(4).map(new OneMapper2()).groupBy(0).sum(1); - - List> result = first.collect(); - - String expected = "(1,1)\n(2,2)\n(3,3)\n(4,4)\n(5,4)\n(6,4)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testFirstNOnGroupedAndSortedDS() throws Exception { - /* - * First-n on grouped and sorted data set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> first = - ds.groupBy(1).sortGroup(0, Order.DESCENDING).first(3).project(1, 0); - - List> result = first.collect(); - - String expected = - "(1,1)\n" - + "(2,3)\n(2,2)\n" - + "(3,6)\n(3,5)\n(3,4)\n" - + "(4,10)\n(4,9)\n(4,8)\n" - + "(5,15)\n(5,14)\n(5,13)\n" - + "(6,21)\n(6,20)\n(6,19)\n"; - - compareResultAsText(result, expected); - } - - /** Test for FLINK-2135. */ - @Test - public void testFaultyCast() throws Exception { - ExecutionEnvironment ee = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet b = ee.fromElements("a", "b"); - GroupReduceOperator a = - b.groupBy( - new KeySelector() { - @Override - public Long getKey(String value) throws Exception { - return 1L; - } - }) - .sortGroup( - new KeySelector() { - @Override - public Double getKey(String value) throws Exception { - return 1.0; - } - }, - Order.DESCENDING) - .first(1); - - List result = b.collect(); - - String expected = "a\nb"; - - compareResultAsText(result, expected); - } - - private static class OneMapper - implements MapFunction, Tuple1> { - private static final long serialVersionUID = 1L; - private final Tuple1 one = new Tuple1(1); - - @Override - public Tuple1 map(Tuple3 value) { - return one; - } - } - - private static class OneMapper2 - implements MapFunction, Tuple2> { - private static final long serialVersionUID = 1L; - private final Tuple2 one = new Tuple2<>(0L, 1); - - @Override - public Tuple2 map(Tuple3 value) { - one.f0 = value.f1; - return one; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java deleted file mode 100644 index 21f38333ffeda..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java +++ /dev/null @@ -1,416 +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.operators; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link FlatMapFunction} and {@link RichFlatMapFunction}. */ -@RunWith(Parameterized.class) -public class FlatMapITCase extends MultipleProgramsTestBaseJUnit4 { - public FlatMapITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testNonPassingFlatMap() throws Exception { - /* - * Test non-passing flatmap - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getStringDataSet(env); - DataSet nonPassingFlatMapDs = ds.flatMap(new FlatMapper1()); - - List result = nonPassingFlatMapDs.collect(); - - String expected = "\n"; - - compareResultAsText(result, expected); - } - - private static class FlatMapper1 implements FlatMapFunction { - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(String value, Collector out) throws Exception { - if (value.contains("bananas")) { - out.collect(value); - } - } - } - - @Test - public void testDataDuplicatingFlatMap() throws Exception { - /* - * Test data duplicating flatmap - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getStringDataSet(env); - DataSet duplicatingFlatMapDs = ds.flatMap(new FlatMapper2()); - - List result = duplicatingFlatMapDs.collect(); - - String expected = - "Hi\n" - + "HI\n" - + "Hello\n" - + "HELLO\n" - + "Hello world\n" - + "HELLO WORLD\n" - + "Hello world, how are you?\n" - + "HELLO WORLD, HOW ARE YOU?\n" - + "I am fine.\n" - + "I AM FINE.\n" - + "Luke Skywalker\n" - + "LUKE SKYWALKER\n" - + "Random comment\n" - + "RANDOM COMMENT\n" - + "LOL\n" - + "LOL\n"; - - compareResultAsText(result, expected); - } - - private static class FlatMapper2 implements FlatMapFunction { - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(String value, Collector out) throws Exception { - out.collect(value); - out.collect(value.toUpperCase()); - } - } - - @Test - public void testFlatMapWithVaryingNumberOfEmittedTuples() throws Exception { - /* - * Test flatmap with varying number of emitted tuples - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> varyingTuplesMapDs = ds.flatMap(new FlatMapper3()); - - List> result = varyingTuplesMapDs.collect(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "2,2,Hello\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "5,3,I am fine.\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "8,4,Comment#2\n" - + "10,4,Comment#4\n" - + "11,5,Comment#5\n" - + "11,5,Comment#5\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "14,5,Comment#8\n" - + "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "17,6,Comment#11\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "20,6,Comment#14\n"; - - compareResultAsTuples(result, expected); - } - - private static class FlatMapper3 - implements FlatMapFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void flatMap( - Tuple3 value, Collector> out) - throws Exception { - final int numTuples = value.f0 % 3; - for (int i = 0; i < numTuples; i++) { - out.collect(value); - } - } - } - - @Test - public void testTypeConversionFlatMapperCustomToTuple() throws Exception { - /* - * Test type conversion flatmapper (Custom -> Tuple) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet> typeConversionFlatMapDs = - ds.flatMap(new FlatMapper4()); - - List> result = typeConversionFlatMapDs.collect(); - - String expected = - "1,0,Hi\n" - + "2,1,Hello\n" - + "2,2,Hello world\n" - + "3,3,Hello world, how are you?\n" - + "3,4,I am fine.\n" - + "3,5,Luke Skywalker\n" - + "4,6,Comment#1\n" - + "4,7,Comment#2\n" - + "4,8,Comment#3\n" - + "4,9,Comment#4\n" - + "5,10,Comment#5\n" - + "5,11,Comment#6\n" - + "5,12,Comment#7\n" - + "5,13,Comment#8\n" - + "5,14,Comment#9\n" - + "6,15,Comment#10\n" - + "6,16,Comment#11\n" - + "6,17,Comment#12\n" - + "6,18,Comment#13\n" - + "6,19,Comment#14\n" - + "6,20,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class FlatMapper4 - implements FlatMapFunction> { - private static final long serialVersionUID = 1L; - private final Tuple3 outTuple = new Tuple3(); - - @Override - public void flatMap(CustomType value, Collector> out) - throws Exception { - outTuple.setField(value.myInt, 0); - outTuple.setField(value.myLong, 1); - outTuple.setField(value.myString, 2); - out.collect(outTuple); - } - } - - @Test - public void testTypeConversionFlatMapperTupleToBasic() throws Exception { - /* - * Test type conversion flatmapper (Tuple -> Basic) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet typeConversionFlatMapDs = ds.flatMap(new FlatMapper5()); - - List result = typeConversionFlatMapDs.collect(); - - String expected = - "Hi\n" - + "Hello\n" - + "Hello world\n" - + "Hello world, how are you?\n" - + "I am fine.\n" - + "Luke Skywalker\n" - + "Comment#1\n" - + "Comment#2\n" - + "Comment#3\n" - + "Comment#4\n" - + "Comment#5\n" - + "Comment#6\n" - + "Comment#7\n" - + "Comment#8\n" - + "Comment#9\n" - + "Comment#10\n" - + "Comment#11\n" - + "Comment#12\n" - + "Comment#13\n" - + "Comment#14\n" - + "Comment#15\n"; - - compareResultAsText(result, expected); - } - - private static class FlatMapper5 - implements FlatMapFunction, String> { - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(Tuple3 value, Collector out) - throws Exception { - out.collect(value.f2); - } - } - - @Test - public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() - throws Exception { - /* - * Test flatmapper if UDF returns input object - * multiple times and changes it in between - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> inputObjFlatMapDs = ds.flatMap(new FlatMapper6()); - - List> result = inputObjFlatMapDs.collect(); - - String expected = - "0,1,Hi\n" - + "0,2,Hello\n" - + "1,2,Hello\n" - + "0,2,Hello world\n" - + "1,2,Hello world\n" - + "2,2,Hello world\n" - + "0,3,I am fine.\n" - + "0,3,Luke Skywalker\n" - + "1,3,Luke Skywalker\n" - + "0,4,Comment#1\n" - + "1,4,Comment#1\n" - + "2,4,Comment#1\n" - + "0,4,Comment#3\n" - + "0,4,Comment#4\n" - + "1,4,Comment#4\n" - + "0,5,Comment#5\n" - + "1,5,Comment#5\n" - + "2,5,Comment#5\n" - + "0,5,Comment#7\n" - + "0,5,Comment#8\n" - + "1,5,Comment#8\n" - + "0,5,Comment#9\n" - + "1,5,Comment#9\n" - + "2,5,Comment#9\n" - + "0,6,Comment#11\n" - + "0,6,Comment#12\n" - + "1,6,Comment#12\n" - + "0,6,Comment#13\n" - + "1,6,Comment#13\n" - + "2,6,Comment#13\n" - + "0,6,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class FlatMapper6 - implements FlatMapFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void flatMap( - Tuple3 value, Collector> out) - throws Exception { - final int numTuples = value.f0 % 4; - for (int i = 0; i < numTuples; i++) { - value.setField(i, 0); - out.collect(value); - } - } - } - - @Test - public void testFlatMapWithBroadcastSet() throws Exception { - /* - * Test flatmap with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ints = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> bcFlatMapDs = - ds.flatMap(new RichFlatMapper1()).withBroadcastSet(ints, "ints"); - List> result = bcFlatMapDs.collect(); - - String expected = - "55,1,Hi\n" - + "55,2,Hello\n" - + "55,2,Hello world\n" - + "55,3,Hello world, how are you?\n" - + "55,3,I am fine.\n" - + "55,3,Luke Skywalker\n" - + "55,4,Comment#1\n" - + "55,4,Comment#2\n" - + "55,4,Comment#3\n" - + "55,4,Comment#4\n" - + "55,5,Comment#5\n" - + "55,5,Comment#6\n" - + "55,5,Comment#7\n" - + "55,5,Comment#8\n" - + "55,5,Comment#9\n" - + "55,6,Comment#10\n" - + "55,6,Comment#11\n" - + "55,6,Comment#12\n" - + "55,6,Comment#13\n" - + "55,6,Comment#14\n" - + "55,6,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class RichFlatMapper1 - extends RichFlatMapFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - private final Tuple3 outTuple = new Tuple3(); - private Integer f2Replace = 0; - - @Override - public void open(OpenContext openContext) { - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - f2Replace = sum; - } - - @Override - public void flatMap( - Tuple3 value, Collector> out) - throws Exception { - outTuple.setFields(f2Replace, value.f1, value.f2); - out.collect(outTuple); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java deleted file mode 100644 index 511af69079356..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java +++ /dev/null @@ -1,552 +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.operators; - -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.Order; -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.operators.UnsortedGrouping; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** - * The GroupCombine operator is not easy to test because it is essentially just a combiner. The - * result can be the result of a normal groupReduce at any stage its execution. The basic idea is to - * preserve the grouping key in the partial result, so that we can do a reduceGroup afterwards to - * finalize the results for verification. In addition, we can use hashPartition to partition the - * data and check if no shuffling (just combining) has been performed. - */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class GroupCombineITCase extends MultipleProgramsTestBaseJUnit4 { - - public GroupCombineITCase(TestExecutionMode mode) { - super(mode); - } - - private static String identityResult = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "6,3,Luke Skywalker\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "9,4,Comment#3\n" - + "10,4,Comment#4\n" - + "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n" - + "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "18,6,Comment#12\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "21,6,Comment#15\n"; - - @Test - public void testAllGroupCombineIdentity() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - DataSet> reduceDs = - ds - // combine - .combineGroup(new IdentityFunction()) - // fully reduce - .reduceGroup(new IdentityFunction()); - - List> result = reduceDs.collect(); - - compareResultAsTuples(result, identityResult); - } - - @Test - public void testIdentity() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - DataSet> reduceDs = - ds - // combine - .combineGroup(new IdentityFunction()) - // fully reduce - .reduceGroup(new IdentityFunction()); - - List> result = reduceDs.collect(); - - compareResultAsTuples(result, identityResult); - } - - @Test - public void testIdentityWithGroupBy() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - DataSet> reduceDs = - ds.groupBy(1) - // combine - .combineGroup(new IdentityFunction()) - // fully reduce - .reduceGroup(new IdentityFunction()); - - List> result = reduceDs.collect(); - - compareResultAsTuples(result, identityResult); - } - - @Test - public void testIdentityWithGroupByAndSort() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - DataSet> reduceDs = - ds.groupBy(1) - .sortGroup(1, Order.DESCENDING) - // reduce partially - .combineGroup(new IdentityFunction()) - .groupBy(1) - .sortGroup(1, Order.DESCENDING) - // fully reduce - .reduceGroup(new IdentityFunction()); - - List> result = reduceDs.collect(); - - compareResultAsTuples(result, identityResult); - } - - @Test - public void testPartialReduceWithIdenticalInputOutputType() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // data - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - DataSet>> dsWrapped = - ds - // wrap values as Kv pairs with the grouping key as key - .map(new Tuple3KvWrapper()); - - List> result = - dsWrapped - .groupBy(0) - // reduce partially - .combineGroup(new Tuple3toTuple3GroupReduce()) - .groupBy(0) - // reduce fully to check result - .reduceGroup(new Tuple3toTuple3GroupReduce()) - // unwrap - .map( - new MapFunction< - Tuple2>, - Tuple3>() { - @Override - public Tuple3 map( - Tuple2> value) - throws Exception { - return value.f1; - } - }) - .collect(); - - String expected = - "1,1,combined\n" - + "5,4,combined\n" - + "15,9,combined\n" - + "34,16,combined\n" - + "65,25,combined\n" - + "111,36,combined\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testPartialReduceWithDifferentInputOutputType() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // data - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - DataSet>> dsWrapped = - ds - // wrap values as Kv pairs with the grouping key as key - .map(new Tuple3KvWrapper()); - - List> result = - dsWrapped - .groupBy(0) - // reduce partially - .combineGroup(new Tuple3toTuple2GroupReduce()) - .groupBy(0) - // reduce fully to check result - .reduceGroup(new Tuple2toTuple2GroupReduce()) - // unwrap - .map( - new MapFunction< - Tuple2>, - Tuple2>() { - @Override - public Tuple2 map( - Tuple2> value) - throws Exception { - return value.f1; - } - }) - .collect(); - - String expected = "1,3\n" + "5,20\n" + "15,58\n" + "34,52\n" + "65,70\n" + "111,96\n"; - - compareResultAsTuples(result, expected); - } - - @Test - // check if no shuffle is being executed - public void testCheckPartitionShuffleGroupBy() throws Exception { - - org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // data - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - // partition and group data - UnsortedGrouping> partitionedDS = - ds.partitionByHash(0).groupBy(1); - - List> result = - partitionedDS - .combineGroup( - new GroupCombineFunction< - Tuple3, Tuple2>() { - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception { - int count = 0; - long key = 0; - for (Tuple3 value : values) { - key = value.f1; - count++; - } - out.collect(new Tuple2<>(key, count)); - } - }) - .collect(); - - String[] localExpected = - new String[] {"(6,6)", "(5,5)" + "(4,4)", "(3,3)", "(2,2)", "(1,1)"}; - - String[] resultAsStringArray = new String[result.size()]; - for (int i = 0; i < resultAsStringArray.length; ++i) { - resultAsStringArray[i] = result.get(i).toString(); - } - Arrays.sort(resultAsStringArray); - - Assert.assertEquals( - "The two arrays were identical.", - false, - Arrays.equals(localExpected, resultAsStringArray)); - } - - @Test - // check if parallelism of 1 results in the same data like a shuffle - public void testCheckPartitionShuffleDOP1() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - env.setParallelism(1); - - // data - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - // partition and group data - UnsortedGrouping> partitionedDS = - ds.partitionByHash(0).groupBy(1); - - List> result = - partitionedDS - .combineGroup( - new GroupCombineFunction< - Tuple3, Tuple2>() { - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception { - int count = 0; - long key = 0; - for (Tuple3 value : values) { - key = value.f1; - count++; - } - out.collect(new Tuple2<>(key, count)); - } - }) - .collect(); - - String expected = "6,6\n" + "5,5\n" + "4,4\n" + "3,3\n" + "2,2\n" + "1,1\n"; - - compareResultAsTuples(result, expected); - } - - @Test - // check if all API methods are callable - public void testAPI() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getStringDataSet(env) - .map( - new MapFunction>() { - @Override - public Tuple1 map(String value) throws Exception { - return new Tuple1<>(value); - } - }); - - // all methods on DataSet - ds.combineGroup(new GroupCombineFunctionExample()) - .output(new DiscardingOutputFormat>()); - - // all methods on UnsortedGrouping - ds.groupBy(0) - .combineGroup(new GroupCombineFunctionExample()) - .output(new DiscardingOutputFormat>()); - - // all methods on SortedGrouping - ds.groupBy(0) - .sortGroup(0, Order.ASCENDING) - .combineGroup(new GroupCombineFunctionExample()) - .output(new DiscardingOutputFormat>()); - - env.execute(); - } - - private static class GroupCombineFunctionExample - implements GroupCombineFunction, Tuple1> { - - @Override - public void combine(Iterable> values, Collector> out) - throws Exception { - for (Tuple1 value : values) { - out.collect(value); - } - } - } - - /** For Scala GroupCombineITCase. */ - public static class ScalaGroupCombineFunctionExample - implements GroupCombineFunction, scala.Tuple1> { - - @Override - public void combine( - Iterable> values, Collector> out) - throws Exception { - for (scala.Tuple1 value : values) { - out.collect(value); - } - } - } - - private static class IdentityFunction - implements GroupCombineFunction< - Tuple3, Tuple3>, - GroupReduceFunction< - Tuple3, Tuple3> { - - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception { - for (Tuple3 value : values) { - out.collect(new Tuple3<>(value.f0, value.f1, value.f2)); - } - } - - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - for (Tuple3 value : values) { - out.collect(new Tuple3<>(value.f0, value.f1, value.f2)); - } - } - } - - private static class Tuple3toTuple3GroupReduce - implements KvGroupReduce< - Long, - Tuple3, - Tuple3, - Tuple3> { - - @Override - public void combine( - Iterable>> values, - Collector>> out) - throws Exception { - int i = 0; - long l = 0; - long key = 0; - - // collapse groups - for (Tuple2> value : values) { - key = value.f0; - Tuple3 extracted = value.f1; - i += extracted.f0; - l += extracted.f1; - } - - Tuple3 result = new Tuple3<>(i, l, "combined"); - out.collect(new Tuple2<>(key, result)); - } - - @Override - public void reduce( - Iterable>> values, - Collector>> out) - throws Exception { - combine(values, out); - } - } - - private static class Tuple3toTuple2GroupReduce - implements KvGroupReduce< - Long, - Tuple3, - Tuple2, - Tuple2> { - - @Override - public void combine( - Iterable>> values, - Collector>> out) - throws Exception { - int i = 0; - long l = 0; - long key = 0; - - // collapse groups - for (Tuple2> value : values) { - key = value.f0; - Tuple3 extracted = value.f1; - i += extracted.f0; - l += extracted.f1 + extracted.f2.length(); - } - - Tuple2 result = new Tuple2<>(i, l); - out.collect(new Tuple2<>(key, result)); - } - - @Override - public void reduce( - Iterable>> values, - Collector>> out) - throws Exception { - new Tuple2toTuple2GroupReduce().reduce(values, out); - } - } - - private static class Tuple2toTuple2GroupReduce - implements KvGroupReduce< - Long, Tuple2, Tuple2, Tuple2> { - - @Override - public void combine( - Iterable>> values, - Collector>> out) - throws Exception { - int i = 0; - long l = 0; - long key = 0; - - // collapse groups - for (Tuple2> value : values) { - key = value.f0; - Tuple2 extracted = value.f1; - i += extracted.f0; - l += extracted.f1; - } - - Tuple2 result = new Tuple2<>(i, l); - - out.collect(new Tuple2<>(key, result)); - } - - @Override - public void reduce( - Iterable>> values, - Collector>> out) - throws Exception { - combine(values, out); - } - } - - private class Tuple3KvWrapper - implements MapFunction< - Tuple3, Tuple2>> { - @Override - public Tuple2> map(Tuple3 value) - throws Exception { - return new Tuple2<>(value.f1, value); - } - } - - private interface CombineAndReduceGroup - extends GroupCombineFunction, GroupReduceFunction {} - - private interface KvGroupReduce - extends CombineAndReduceGroup, Tuple2, Tuple2> {} -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java deleted file mode 100644 index 042fa97e61071..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java +++ /dev/null @@ -1,1757 +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.operators; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.operators.Order; -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.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CrazyNested; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.operators.util.CollectionDataSets.FromTupleWithCTor; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.operators.util.CollectionDataSets.PojoContainingTupleAndWritable; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.joda.time.DateTime; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.Date; -import java.util.Iterator; -import java.util.List; - -import scala.math.BigInt; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** - * Integration tests for {@link GroupReduceFunction}, {@link RichGroupReduceFunction}, and {@link - * GroupCombineFunction}. - */ -@SuppressWarnings({"serial", "unchecked", "UnusedDeclaration"}) -@RunWith(Parameterized.class) -public class GroupReduceITCase extends MultipleProgramsTestBaseJUnit4 { - - public GroupReduceITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void - testCorrectnessofGroupReduceOnTupleContainingPrimitiveByteArrayWithKeyFieldSelectors() - throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getTuple2WithByteArrayDataSet(env); - DataSet reduceDs = ds.groupBy(0).reduceGroup(new ByteArrayGroupReduce()); - - List result = reduceDs.collect(); - - String expected = "0\n" + "1\n" + "2\n" + "3\n" + "4\n"; - - compareResultAsText(result, expected); - } - - private static class ByteArrayGroupReduce - implements GroupReduceFunction, Integer> { - @Override - public void reduce(Iterable> values, Collector out) - throws Exception { - int sum = 0; - for (Tuple2 value : values) { - sum += value.f1; - } - out.collect(sum); - } - } - - @Test - public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelector() throws Exception { - /* - * check correctness of groupReduce on tuples with key field selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1).reduceGroup(new Tuple3GroupReduce()); - - List> result = reduceDs.collect(); - - String expected = "1,1\n" + "5,2\n" + "15,3\n" + "34,4\n" + "65,5\n" + "111,6\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() - throws Exception { - /* - * check correctness of groupReduce on tuples with multiple key field selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(4, 0).reduceGroup(new Tuple5GroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,P-),1\n" - + "2,3,0,P-),1\n" - + "2,2,0,P-),2\n" - + "3,9,0,P-),2\n" - + "3,6,0,P-),3\n" - + "4,17,0,P-),1\n" - + "4,17,0,P-),2\n" - + "5,11,0,P-),1\n" - + "5,29,0,P-),2\n" - + "5,25,0,P-),3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSorting() - throws Exception { - /* - * check correctness of groupReduce on tuples with key field selector and group sorting - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1) - .sortGroup(2, Order.ASCENDING) - .reduceGroup(new Tuple3SortedGroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "5,2,Hello-Hello world\n" - + "15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" - + "34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" - + "65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" - + "111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceOnTuplesWithKeyExtractor() throws Exception { - /* - * check correctness of groupReduce on tuples with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new KeySelector1()).reduceGroup(new Tuple3GroupReduce()); - - List> result = reduceDs.collect(); - - String expected = "1,1\n" + "5,2\n" + "15,3\n" + "34,4\n" + "65,5\n" + "111,6\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector1 implements KeySelector, Long> { - private static final long serialVersionUID = 1L; - - @Override - public Long getKey(Tuple3 in) { - return in.f1; - } - } - - @Test - public void testCorrectnessOfGroupReduceOnCustomTypeWithTypeExtractor() throws Exception { - /* - * check correctness of groupReduce on custom type with type extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = - ds.groupBy(new KeySelector2()).reduceGroup(new CustomTypeGroupReduce()); - - List result = reduceDs.collect(); - - String expected = - "1,0,Hello!\n" - + "2,3,Hello!\n" - + "3,12,Hello!\n" - + "4,30,Hello!\n" - + "5,60,Hello!\n" - + "6,105,Hello!\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector2 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testCorrectnessOfAllGroupReduceForTuples() throws Exception { - /* - * check correctness of all-groupreduce for tuples - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.reduceGroup(new AllAddingTuple3GroupReduce()); - - List> result = reduceDs.collect(); - - String expected = "231,91,Hello World\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfAllGroupReduceForCustomTypes() throws Exception { - /* - * check correctness of all-groupreduce for custom types - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = ds.reduceGroup(new AllAddingCustomTypeGroupReduce()); - - List result = reduceDs.collect(); - - String expected = "91,210,Hello!"; - - compareResultAsText(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceWithBroadcastSet() throws Exception { - /* - * check correctness of groupReduce with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1) - .reduceGroup(new BCTuple3GroupReduce()) - .withBroadcastSet(intDs, "ints"); - - List> result = reduceDs.collect(); - - String expected = - "1,1,55\n" + "5,2,55\n" + "15,3,55\n" + "34,4,55\n" + "65,5,55\n" + "111,6,55\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceIfUDFReturnsInputObjectsMultipleTimesWhileChangingThem() - throws Exception { - /* - * check correctness of groupReduce if UDF returns input objects multiple times and changes it in between - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1).reduceGroup(new InputReturningTuple3GroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "11,1,Hi!\n" - + "21,1,Hi again!\n" - + "12,2,Hi!\n" - + "22,2,Hi again!\n" - + "13,2,Hi!\n" - + "23,2,Hi again!\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine() - throws Exception { - /* - * check correctness of groupReduce on custom type with key extractor and combine - */ - org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = - ds.groupBy(new KeySelector3()).reduceGroup(new CustomTypeGroupReduceWithCombine()); - - List result = reduceDs.collect(); - - String expected = - "1,0,test1\n" - + "2,3,test2\n" - + "3,12,test3\n" - + "4,30,test4\n" - + "5,60,test5\n" - + "6,105,test6\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector3 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testCorrectnessOfGroupReduceOnTuplesWithCombine() throws Exception { - /* - * check correctness of groupReduce on tuples with combine - */ - org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); // important because it determines how often the combiner is called - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine()); - - List> result = reduceDs.collect(); - - String expected = - "1,test1\n" - + "5,test2\n" - + "15,test3\n" - + "34,test4\n" - + "65,test5\n" - + "111,test6\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfAllGroupReduceForTuplesWithCombine() throws Exception { - /* - * check correctness of all-groupreduce for tuples with combine - */ - org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION); - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get3TupleDataSet(env) - .map(new IdentityMapper>()) - .setParallelism(4); - - Configuration cfg = new Configuration(); - cfg.setString(Optimizer.HINT_SHIP_STRATEGY, Optimizer.HINT_SHIP_STRATEGY_REPARTITION); - DataSet> reduceDs = - ds.reduceGroup(new Tuple3AllGroupReduceWithCombine()).withParameters(cfg); - - List> result = reduceDs.collect(); - - String expected = - "322," - + "testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupreduceWithDescendingGroupSort() throws Exception { - /* - * check correctness of groupReduce with descending group sort - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1) - .sortGroup(2, Order.DESCENDING) - .reduceGroup(new Tuple3SortedGroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "5,2,Hello world-Hello\n" - + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" - + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" - + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" - + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() - throws Exception { - /* - * check correctness of groupReduce on tuples with tuple-returning key selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new KeySelector4()).reduceGroup(new Tuple5GroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,P-),1\n" - + "2,3,0,P-),1\n" - + "2,2,0,P-),2\n" - + "3,9,0,P-),2\n" - + "3,6,0,P-),3\n" - + "4,17,0,P-),1\n" - + "4,17,0,P-),2\n" - + "5,11,0,P-),1\n" - + "5,29,0,P-),2\n" - + "5,25,0,P-),3\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector4 - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 t) { - return new Tuple2<>(t.f0, t.f4); - } - } - - @Test - public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting() - throws Exception { - /* - * check that input of combiner is also sorted for combinable groupReduce with group sorting - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1) - .sortGroup(0, Order.ASCENDING) - .reduceGroup(new OrderCheckingCombinableReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "4,3,Hello world, how are you?\n" - + "7,4,Comment#1\n" - + "11,5,Comment#5\n" - + "16,6,Comment#10\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testDeepNesting() throws Exception { - /* - * Deep nesting test - * + null value in pojo - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCrazyNestedDataSet(env); - DataSet> reduceDs = - ds.groupBy("nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal") - .reduceGroup(new GroupReducer1()); - - List> result = reduceDs.collect(); - - String expected = "aa,1\nbb,2\ncc,3\n"; - - compareResultAsTuples(result, expected); - } - - private static class GroupReducer1 - implements GroupReduceFunction< - CollectionDataSets.CrazyNested, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector> out) - throws Exception { - int c = 0; - String n = null; - for (CrazyNested v : values) { - c++; // haha - n = v.nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal; - } - out.collect(new Tuple2<>(n, c)); - } - } - - @Test - public void testPojoExtendingFromTupleWithCustomFields() throws Exception { - /* - * Test Pojo extending from tuple WITH custom fields - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getPojoExtendingFromTuple(env); - DataSet reduceDs = ds.groupBy("special", "f2").reduceGroup(new GroupReducer2()); - - List result = reduceDs.collect(); - - String expected = "3\n2\n"; - - compareResultAsText(result, expected); - } - - private static class GroupReducer2 implements GroupReduceFunction { - - @Override - public void reduce(Iterable values, Collector out) { - out.collect(countElements(values)); - } - } - - @Test - public void testPojoContainigWritableAndTuples() throws Exception { - /* - * Test Pojo containing a Writable and Tuples - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = - CollectionDataSets.getPojoContainingTupleAndWritable(env); - DataSet reduceDs = - ds.groupBy("hadoopFan", "theTuple.*") // full tuple selection - .reduceGroup(new GroupReducer3()); - - List result = reduceDs.collect(); - - String expected = "1\n5\n"; - - compareResultAsText(result, expected); - } - - private static class GroupReducer3 - implements GroupReduceFunction { - - @Override - public void reduce( - Iterable values, Collector out) { - out.collect(countElements(values)); - } - } - - @Test - public void testTupleContainingPojosAndRegularFields() throws Exception { - /* - * Test Tuple containing pojos and regular fields - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.getTupleContainingPojos(env); - - DataSet reduceDs = - ds.groupBy("f0", "f1.*") // nested full tuple selection - .reduceGroup(new GroupReducer4()); - - List result = reduceDs.collect(); - - String expected = "3\n1\n"; - - compareResultAsText(result, expected); - } - - private static class GroupReducer4 - implements GroupReduceFunction, Integer> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, Collector out) { - out.collect(countElements(values)); - } - } - - @Test - public void testStringBasedDefinitionOnGroupSort() throws Exception { - /* - * Test string-based definition on group sort, based on test: - * check correctness of groupReduce with descending group sort - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1) - .sortGroup("f2", Order.DESCENDING) - .reduceGroup(new Tuple3SortedGroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "5,2,Hello world-Hello\n" - + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" - + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" - + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" - + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testIntBasedDefinitionOnGroupSortForFullNestedTuple() throws Exception { - /* - * Test int-based definition on group sort, for (full) nested Tuple - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet, String>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet(env); - DataSet reduceDs = - ds.groupBy("f1") - .sortGroup(0, Order.DESCENDING) - .reduceGroup(new NestedTupleReducer()); - List result = reduceDs.collect(); - - String expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n" + "c--(4,9)-(3,6)-(3,3)-\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testIntBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exception { - /* - * Test int-based definition on group sort, for (partial) nested Tuple ASC - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet, String>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet(env); - // f0.f0 is first integer - DataSet reduceDs = - ds.groupBy("f1") - .sortGroup("f0.f0", Order.ASCENDING) - .sortGroup("f0.f1", Order.ASCENDING) - .reduceGroup(new NestedTupleReducer()); - List result = reduceDs.collect(); - - String expected = "a--(1,2)-(1,3)-(2,1)-\n" + "b--(2,2)-\n" + "c--(3,3)-(3,6)-(4,9)-\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testStringBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exception { - /* - * Test string-based definition on group sort, for (partial) nested Tuple DESC - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet, String>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet(env); - // f0.f0 is first integer - DataSet reduceDs = - ds.groupBy("f1") - .sortGroup("f0.f0", Order.DESCENDING) - .reduceGroup(new NestedTupleReducer()); - List result = reduceDs.collect(); - - String expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n" + "c--(4,9)-(3,3)-(3,6)-\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeys() throws Exception { - /* - * Test string-based definition on group sort, for two grouping keys - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet, String>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet(env); - // f0.f0 is first integer - DataSet reduceDs = - ds.groupBy("f1") - .sortGroup("f0.f0", Order.DESCENDING) - .sortGroup("f0.f1", Order.DESCENDING) - .reduceGroup(new NestedTupleReducer()); - List result = reduceDs.collect(); - - String expected = "a--(2,1)-(1,3)-(1,2)-\n" + "b--(2,2)-\n" + "c--(4,9)-(3,6)-(3,3)-\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeysWithPojos() throws Exception { - /* - * Test string-based definition on group sort, for two grouping keys with Pojos - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet ds = - CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env); - // f0.f0 is first integer - DataSet reduceDs = - ds.groupBy("hadoopFan") - .sortGroup("theTuple.f0", Order.DESCENDING) - .sortGroup("theTuple.f1", Order.DESCENDING) - .reduceGroup(new GroupReducer5()); - List result = reduceDs.collect(); - - String expected = "1---(10,100)-\n" + "2---(30,600)-(30,400)-(30,200)-(20,201)-(20,200)-\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testTupleKeySelectorGroupSort() throws Exception { - /* - * check correctness of sorted groupReduce on tuples with keyselector sorting - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new LongFieldExtractor>(1)) - .sortGroup( - new StringFieldExtractor>(2), - Order.DESCENDING) - .reduceGroup(new Tuple3SortedGroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "5,2,Hello world-Hello\n" - + "15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" - + "34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" - + "65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" - + "111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - - compareResultAsTuples(result, expected); - } - - private static class TwoTuplePojoExtractor - implements KeySelector> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(CustomType value) throws Exception { - return new Tuple2<>(value.myInt, value.myInt); - } - } - - private static class StringPojoExtractor implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public String getKey(CustomType value) throws Exception { - return value.myString; - } - } - - @Test - public void testPojoKeySelectorGroupSort() throws Exception { - /* - * check correctness of sorted groupReduce on custom type with keyselector sorting - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = - ds.groupBy(new TwoTuplePojoExtractor()) - .sortGroup(new StringPojoExtractor(), Order.DESCENDING) - .reduceGroup(new CustomTypeSortedGroupReduce()); - - List result = reduceDs.collect(); - - String expected = - "1,0,Hi\n" - + "2,3,Hello world-Hello\n" - + "3,12,Luke Skywalker-I am fine.-Hello world, how are you?\n" - + "4,30,Comment#4-Comment#3-Comment#2-Comment#1\n" - + "5,60,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" - + "6,105,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - - compareResultAsText(result, expected); - } - - private static class LongFieldExtractor implements KeySelector { - private static final long serialVersionUID = 1L; - private int field; - - public LongFieldExtractor() {} - - public LongFieldExtractor(int field) { - this.field = field; - } - - @Override - public Long getKey(T t) throws Exception { - return ((Tuple) t).getField(field); - } - } - - private static class IntFieldExtractor implements KeySelector { - private static final long serialVersionUID = 1L; - private int field; - - public IntFieldExtractor() {} - - public IntFieldExtractor(int field) { - this.field = field; - } - - @Override - public Integer getKey(T t) throws Exception { - return ((Tuple) t).getField(field); - } - } - - private static class StringFieldExtractor implements KeySelector { - private static final long serialVersionUID = 1L; - private int field; - - public StringFieldExtractor() {} - - public StringFieldExtractor(int field) { - this.field = field; - } - - @Override - public String getKey(T t) throws Exception { - return t.getField(field); - } - } - - @Test - public void testTupleKeySelectorSortWithCombine() throws Exception { - /* - * check correctness of sorted groupReduce with combine on tuples with keyselector sorting - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new LongFieldExtractor>(1)) - .sortGroup( - new StringFieldExtractor>(2), - Order.DESCENDING) - .reduceGroup(new Tuple3SortedGroupReduceWithCombine()); - - List> result = reduceDs.collect(); - - if (super.mode != TestExecutionMode.COLLECTION) { - String expected = - "1,Hi\n" - + "5,Hello world-Hello\n" - + "15,Luke Skywalker-I am fine.-Hello world, how are you?\n" - + "34,Comment#4-Comment#3-Comment#2-Comment#1\n" - + "65,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" - + "111,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n"; - - compareResultAsTuples(result, expected); - } - } - - private static class FiveToTwoTupleExtractor - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 in) { - return new Tuple2<>(in.f4, in.f2); - } - } - - @Test - public void testTupleKeySelectorSortCombineOnTuple() throws Exception { - /* - * check correctness of sorted groupReduceon with Tuple2 keyselector sorting - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new IntFieldExtractor>(0)) - .sortGroup(new FiveToTwoTupleExtractor(), Order.DESCENDING) - .reduceGroup(new Tuple5SortedGroupReduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "2,5,0,Hallo Welt-Hallo Welt wie,1\n" - + "3,15,0,BCD-ABC-Hallo Welt wie gehts?,2\n" - + "4,34,0,FGH-CDE-EFG-DEF,1\n" - + "5,65,0,IJK-HIJ-KLM-JKL-GHI,1\n"; - - compareResultAsTuples(result, expected); - } - - private static class GroupReducer5 - implements GroupReduceFunction< - CollectionDataSets.PojoContainingTupleAndWritable, String> { - @Override - public void reduce(Iterable values, Collector out) - throws Exception { - boolean once = false; - StringBuilder concat = new StringBuilder(); - for (PojoContainingTupleAndWritable value : values) { - if (!once) { - concat.append(value.hadoopFan.get()); - concat.append("---"); - once = true; - } - concat.append(value.theTuple); - concat.append("-"); - } - out.collect(concat.toString()); - } - } - - @Test - public void testGroupingWithPojoContainingMultiplePojos() throws Exception { - /* - * Test grouping with pojo containing multiple pojos (was a bug) - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet ds = - CollectionDataSets.getPojoWithMultiplePojos(env); - - // f0.f0 is first integer - DataSet reduceDs = ds.groupBy("p2.a2").reduceGroup(new GroupReducer6()); - List result = reduceDs.collect(); - - String expected = "b\nccc\nee\n"; - - compareResultAsText(result, expected); - } - - private static class GroupReducer6 - implements GroupReduceFunction { - @Override - public void reduce( - Iterable values, Collector out) - throws Exception { - StringBuilder concat = new StringBuilder(); - for (CollectionDataSets.PojoWithMultiplePojos value : values) { - concat.append(value.p2.a2); - } - out.collect(concat.toString()); - } - } - - @Test - public void testJavaCollectionsWithinPojos() throws Exception { - /* - * Test Java collections within pojos ( == test kryo) - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet ds = - CollectionDataSets.getPojoWithCollection(env); - // f0.f0 is first integer - DataSet reduceDs = ds.groupBy("key").reduceGroup(new GroupReducer7()); - List result = reduceDs.collect(); - - String expected = - "callFor key 0 we got: pojo.a=apojo.a=bFor key 0 we got: pojo.a=a2pojo.a=b2\n"; - - compareResultAsText(result, expected); - } - - private static class GroupReducer7 - implements GroupReduceFunction { - - @Override - public void reduce( - Iterable values, Collector out) { - StringBuilder concat = new StringBuilder(); - concat.append("call"); - for (CollectionDataSets.PojoWithCollection value : values) { - concat.append("For key ").append(value.key).append(" we got: "); - - for (CollectionDataSets.Pojo1 p : value.pojos) { - concat.append("pojo.a=").append(p.a); - } - } - out.collect(concat.toString()); - } - } - - @Test - public void testGroupByGenericType() throws Exception { - /* - * Group by generic type - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet ds = - CollectionDataSets.getPojoWithCollection(env); - - // f0.f0 is first integer - DataSet reduceDs = ds.groupBy("bigInt").reduceGroup(new GroupReducer8()); - List result = reduceDs.collect(); - ExecutionConfig ec = env.getConfig(); - - // check if automatic type registration with Kryo worked - Assert.assertTrue(ec.getSerializerConfig().getRegisteredKryoTypes().contains(BigInt.class)); - Assert.assertFalse( - ec.getSerializerConfig().getRegisteredKryoTypes().contains(java.sql.Date.class)); - - String expected = null; - - String localExpected = - "[call\n" - + "For key 92233720368547758070 we got:\n" - + "PojoWithCollection{pojos.size()=2, key=0, sqlDate=2033-05-18, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=10, mixed=[{someKey=1}, /this/is/wrong, uhlala]}\n" - + "For key 92233720368547758070 we got:\n" - + "PojoWithCollection{pojos.size()=2, key=0, sqlDate=1976-05-03, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=31104000, mixed=null}]"; - - Assert.assertEquals(localExpected, result.toString()); - } - - @Test - public void testGroupReduceSelectorKeysWithSemProps() throws Exception { - - /* - * Test that semantic properties are correctly adapted when using Selector Keys - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds - // group by selector key - .groupBy( - new KeySelector< - Tuple5, Long>() { - @Override - public Long getKey( - Tuple5 v) - throws Exception { - return (v.f0 * v.f1) - (v.f2 * v.f4); - } - }) - .reduceGroup( - new GroupReduceFunction< - Tuple5, - Tuple5>() { - @Override - public void reduce( - Iterable> - values, - Collector> - out) - throws Exception { - for (Tuple5 v : - values) { - out.collect(v); - } - } - }) - // add forward field information - .withForwardedFields("0") - // group again and reduce - .groupBy(0) - .reduceGroup( - new GroupReduceFunction< - Tuple5, - Tuple2>() { - @Override - public void reduce( - Iterable> - values, - Collector> out) - throws Exception { - int k = 0; - long s = 0; - for (Tuple5 v : - values) { - k = v.f0; - s += v.f1; - } - out.collect(new Tuple2<>(k, s)); - } - }); - - List> result = reduceDs.collect(); - - String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testGroupReduceWithAtomicValue() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds = env.fromElements(1, 1, 2, 3, 4); - DataSet reduceDs = - ds.groupBy("*") - .reduceGroup( - new GroupReduceFunction() { - @Override - public void reduce( - Iterable values, Collector out) - throws Exception { - out.collect(values.iterator().next()); - } - }); - - List result = reduceDs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4"; - - compareResultAsText(result, expected); - } - - /** - * Fix for FLINK-2019. - * - * @throws Exception - */ - @Test - public void testJodatimeDateTimeWithKryo() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds = env.fromElements(new Tuple2<>(1, DateTime.now())); - DataSet> reduceDs = ds.groupBy("f1").sum(0).project(0); - - List> result = reduceDs.collect(); - - String expected = "1\n"; - - compareResultAsTuples(result, expected); - } - - /** - * Fix for FLINK-2158. - * - * @throws Exception - */ - @Test - public void testDateNullException() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> in = - env.fromElements( - new Tuple2<>(0, new Date(1230000000)), - new Tuple2(1, null), - new Tuple2<>(2, new Date(1230000000))); - - DataSet r = - in.groupBy(0) - .reduceGroup( - new GroupReduceFunction, String>() { - @Override - public void reduce( - Iterable> values, - Collector out) - throws Exception { - for (Tuple2 e : values) { - out.collect(Integer.toString(e.f0)); - } - } - }); - - List result = r.collect(); - - String expected = "0\n1\n2\n"; - compareResultAsText(result, expected); - } - - private static class GroupReducer8 - implements GroupReduceFunction { - @Override - public void reduce( - Iterable values, Collector out) { - StringBuilder concat = new StringBuilder(); - concat.append("call"); - for (CollectionDataSets.PojoWithCollection value : values) { - concat.append("\nFor key ").append(value.bigInt).append(" we got:\n").append(value); - } - out.collect(concat.toString()); - } - } - - private static class NestedTupleReducer - implements GroupReduceFunction, String>, String> { - @Override - public void reduce( - Iterable, String>> values, Collector out) { - boolean once = false; - StringBuilder concat = new StringBuilder(); - for (Tuple2, String> value : values) { - if (!once) { - concat.append(value.f1).append("--"); - once = true; - } - concat.append(value.f0); // the tuple with the sorted groups - concat.append("-"); - } - out.collect(concat.toString()); - } - } - - private static class Tuple3GroupReduce - implements GroupReduceFunction, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) { - int i = 0; - long l = 0L; - - for (Tuple3 t : values) { - i += t.f0; - l = t.f1; - } - - out.collect(new Tuple2<>(i, l)); - } - } - - private static class Tuple3SortedGroupReduce - implements GroupReduceFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) { - int sum = 0; - long key = 0; - StringBuilder concat = new StringBuilder(); - - for (Tuple3 next : values) { - sum += next.f0; - key = next.f1; - concat.append(next.f2).append("-"); - } - - if (concat.length() > 0) { - concat.setLength(concat.length() - 1); - } - - out.collect(new Tuple3<>(sum, key, concat.toString())); - } - } - - private static class Tuple5GroupReduce - implements GroupReduceFunction< - Tuple5, - Tuple5> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) { - int i = 0; - long l = 0L; - long l2 = 0L; - - for (Tuple5 t : values) { - i = t.f0; - l += t.f1; - l2 = t.f4; - } - - out.collect(new Tuple5<>(i, l, 0, "P-)", l2)); - } - } - - private static class Tuple5SortedGroupReduce - implements GroupReduceFunction< - Tuple5, - Tuple5> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) { - int i = 0; - long l = 0L; - long l2 = 0L; - StringBuilder concat = new StringBuilder(); - - for (Tuple5 t : values) { - i = t.f0; - l += t.f1; - concat.append(t.f3).append("-"); - l2 = t.f4; - } - if (concat.length() > 0) { - concat.setLength(concat.length() - 1); - } - - out.collect(new Tuple5<>(i, l, 0, concat.toString(), l2)); - } - } - - private static class CustomTypeGroupReduce - implements GroupReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - final Iterator iter = values.iterator(); - - CustomType o = new CustomType(); - CustomType c = iter.next(); - - o.myString = "Hello!"; - o.myInt = c.myInt; - o.myLong = c.myLong; - - while (iter.hasNext()) { - CustomType next = iter.next(); - o.myLong += next.myLong; - } - - out.collect(o); - } - } - - private static class CustomTypeSortedGroupReduce - implements GroupReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - final Iterator iter = values.iterator(); - - CustomType o = new CustomType(); - CustomType c = iter.next(); - - StringBuilder concat = new StringBuilder(c.myString); - o.myInt = c.myInt; - o.myLong = c.myLong; - - while (iter.hasNext()) { - CustomType next = iter.next(); - concat.append("-").append(next.myString); - o.myLong += next.myLong; - } - - o.myString = concat.toString(); - out.collect(o); - } - } - - private static class InputReturningTuple3GroupReduce - implements GroupReduceFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) { - - for (Tuple3 t : values) { - - if (t.f0 < 4) { - t.f2 = "Hi!"; - t.f0 += 10; - out.collect(t); - t.f0 += 10; - t.f2 = "Hi again!"; - out.collect(t); - } - } - } - } - - private static class AllAddingTuple3GroupReduce - implements GroupReduceFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) { - - int i = 0; - long l = 0L; - - for (Tuple3 t : values) { - i += t.f0; - l += t.f1; - } - - out.collect(new Tuple3<>(i, l, "Hello World")); - } - } - - private static class AllAddingCustomTypeGroupReduce - implements GroupReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) { - - CustomType o = new CustomType(0, 0, "Hello!"); - - for (CustomType next : values) { - o.myInt += next.myInt; - o.myLong += next.myLong; - } - - out.collect(o); - } - } - - private static class BCTuple3GroupReduce - extends RichGroupReduceFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - private String f2Replace = ""; - - @Override - public void open(OpenContext openContext) { - - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - f2Replace = sum + ""; - } - - @Override - public void reduce( - Iterable> values, - Collector> out) { - - int i = 0; - long l = 0L; - - for (Tuple3 t : values) { - i += t.f0; - l = t.f1; - } - - out.collect(new Tuple3<>(i, l, this.f2Replace)); - } - } - - private static class Tuple3GroupReduceWithCombine - implements GroupReduceFunction, Tuple2>, - GroupCombineFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void combine( - Iterable> values, - Collector> out) { - - Tuple3 o = new Tuple3<>(0, 0L, ""); - - for (Tuple3 t : values) { - o.f0 += t.f0; - o.f1 = t.f1; - o.f2 = "test" + o.f1; - } - - out.collect(o); - } - - @Override - public void reduce( - Iterable> values, - Collector> out) { - - int i = 0; - String s = ""; - - for (Tuple3 t : values) { - i += t.f0; - s = t.f2; - } - - out.collect(new Tuple2<>(i, s)); - } - } - - private static class Tuple3SortedGroupReduceWithCombine - implements GroupReduceFunction, Tuple2>, - GroupCombineFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void combine( - Iterable> values, - Collector> out) { - int sum = 0; - long key = 0; - StringBuilder concat = new StringBuilder(); - - for (Tuple3 next : values) { - sum += next.f0; - key = next.f1; - concat.append(next.f2).append("-"); - } - - if (concat.length() > 0) { - concat.setLength(concat.length() - 1); - } - - out.collect(new Tuple3<>(sum, key, concat.toString())); - } - - @Override - public void reduce( - Iterable> values, - Collector> out) { - int i = 0; - String s = ""; - - for (Tuple3 t : values) { - i += t.f0; - s = t.f2; - } - - out.collect(new Tuple2<>(i, s)); - } - } - - private static class Tuple3AllGroupReduceWithCombine - implements GroupReduceFunction, Tuple2>, - GroupCombineFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void combine( - Iterable> values, - Collector> out) { - - Tuple3 o = new Tuple3<>(0, 0L, ""); - - for (Tuple3 t : values) { - o.f0 += t.f0; - o.f1 += t.f1; - o.f2 += "test"; - } - - out.collect(o); - } - - @Override - public void reduce( - Iterable> values, - Collector> out) { - - int i = 0; - String s = ""; - - for (Tuple3 t : values) { - i += t.f0 + t.f1; - s += t.f2; - } - - out.collect(new Tuple2<>(i, s)); - } - } - - private static class CustomTypeGroupReduceWithCombine - implements GroupReduceFunction, - GroupCombineFunction { - private static final long serialVersionUID = 1L; - - @Override - public void combine(Iterable values, Collector out) - throws Exception { - - CustomType o = new CustomType(); - - for (CustomType c : values) { - o.myInt = c.myInt; - o.myLong += c.myLong; - o.myString = "test" + c.myInt; - } - - out.collect(o); - } - - @Override - public void reduce(Iterable values, Collector out) { - - CustomType o = new CustomType(0, 0, ""); - - for (CustomType c : values) { - o.myInt = c.myInt; - o.myLong += c.myLong; - o.myString = c.myString; - } - - out.collect(o); - } - } - - private static class OrderCheckingCombinableReduce - implements GroupReduceFunction< - Tuple3, Tuple3>, - GroupCombineFunction< - Tuple3, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - Iterator> it = values.iterator(); - Tuple3 t = it.next(); - - int i = t.f0; - out.collect(t); - - while (it.hasNext()) { - t = it.next(); - if (i > t.f0 || t.f2.equals("INVALID-ORDER!")) { - t.f2 = "INVALID-ORDER!"; - out.collect(t); - } - } - } - - @Override - public void combine( - Iterable> values, - Collector> out) { - - Iterator> it = values.iterator(); - Tuple3 t = it.next(); - - int i = t.f0; - out.collect(t); - - while (it.hasNext()) { - t = it.next(); - if (i > t.f0) { - t.f2 = "INVALID-ORDER!"; - out.collect(t); - } - } - } - } - - private static final class IdentityMapper extends RichMapFunction { - @Override - public T map(T value) { - return value; - } - } - - private static int countElements(Iterable iterable) { - int c = 0; - for (@SuppressWarnings("unused") Object o : iterable) { - c++; - } - return c; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java deleted file mode 100644 index f5d90f07b991f..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java +++ /dev/null @@ -1,1032 +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.operators; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.distributions.DataDistribution; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFlatJoinFunction; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -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.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple6; -import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.IOException; -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link JoinFunction} and {@link FlatJoinFunction}. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class JoinITCase extends MultipleProgramsTestBaseJUnit4 { - - public JoinITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testUDFJoinOnTuplesWithKeyFieldPositions() throws Exception { - /* - * UDF Join on tuples with key field positions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2).where(1).equalTo(1).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testeUDFJoinOnTuplesWithMultipleKeyFieldPositions() throws Exception { - /* - * UDF Join on tuples with multiple key field positions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.get3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2).where(0, 1).equalTo(0, 4).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "Hello world,Hallo Welt wie gehts?\n" - + "Hello world,ABC\n" - + "I am fine.,HIJ\n" - + "I am fine.,IJK\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testDefaultJoinOnTuples() throws Exception { - /* - * Default Join on tuples - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet, Tuple5>> - joinDs = ds1.join(ds2).where(0).equalTo(2); - - List, Tuple5>> - result = joinDs.collect(); - - String expected = - "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" - + "(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" - + "(3,2,Hello world),(3,4,3,Hallo Welt wie gehts?,2)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithHuge() throws Exception { - /* - * Join with Huge - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.joinWithHuge(ds2).where(1).equalTo(1).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithTiny() throws Exception { - /* - * Join with Tiny - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.joinWithTiny(ds2).where(1).equalTo(1).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinThatReturnsTheLeftInputObject() throws Exception { - /* - * Join that returns the left input object - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2).where(1).equalTo(1).with(new LeftReturningJoin()); - - List> result = joinDs.collect(); - - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinThatReturnsTheRightInputObject() throws Exception { - /* - * Join that returns the right input object - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2).where(1).equalTo(1).with(new RightReturningJoin()); - - List> result = joinDs.collect(); - - String expected = "1,1,0,Hallo,1\n" + "2,2,1,Hallo Welt,2\n" + "2,2,1,Hallo Welt,2\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithBroadcastSet() throws Exception { - /* - * Join with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds1 = CollectionDataSets.get3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2) - .where(1) - .equalTo(4) - .with(new T3T5BCJoin()) - .withBroadcastSet(intDs, "ints"); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo,55\n" - + "Hi,Hallo Welt wie,55\n" - + "Hello,Hallo Welt,55\n" - + "Hello world,Hallo Welt,55\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyFieldSelector() - throws Exception { - /* - * Join on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet> ds2 = CollectionDataSets.get3TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2).where(new KeySelector1()).equalTo(0).with(new CustT3Join()); - - List> result = joinDs.collect(); - - String expected = "Hi,Hi\n" + "Hello,Hello\n" + "Hello world,Hello\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector1 implements KeySelector { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - } - - @Test - public void testProjectOnATuple1Input() throws Exception { - /* - * Project join on a tuple input 1 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2) - .where(1) - .equalTo(1) - .projectFirst(2, 1) - .projectSecond(3) - .projectFirst(0) - .projectSecond(4, 1); - - List> result = joinDs.collect(); - - String expected = - "Hi,1,Hallo,1,1,1\n" - + "Hello,2,Hallo Welt,2,2,2\n" - + "Hello world,2,Hallo Welt,3,2,2\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testProjectJoinOnATuple2Input() throws Exception { - /* - * Project join on a tuple input 2 - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2) - .where(1) - .equalTo(1) - .projectSecond(3) - .projectFirst(2, 1) - .projectSecond(4, 1) - .projectFirst(0); - - List> result = joinDs.collect(); - - String expected = - "Hallo,Hi,1,1,1,1\n" - + "Hallo Welt,Hello,2,2,2,2\n" - + "Hallo Welt,Hello world,2,2,2,3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor() - throws Exception { - /* - * Join on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet ds2 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet> joinDs = - ds1.join(ds2).where(1).equalTo(new KeySelector2()).with(new T3CustJoin()); - - List> result = joinDs.collect(); - - String expected = "Hi,Hello\n" + "Hello,Hello world\n" + "Hello world,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector2 implements KeySelector { - @Override - public Long getKey(CustomType value) { - return value.myLong; - } - } - - @Test - public void testDefaultJoinOnTwoCustomTypeInputsWithKeyExtractors() throws Exception { - /* - * (Default) Join on two custom type inputs with key extractors - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); - - DataSet> joinDs = - ds1.join(ds2).where(new KeySelector5()).equalTo(new KeySelector6()); - - List> result = joinDs.collect(); - - String expected = - "1,0,Hi,1,0,Hi\n" - + "2,1,Hello,2,1,Hello\n" - + "2,1,Hello,2,2,Hello world\n" - + "2,2,Hello world,2,1,Hello\n" - + "2,2,Hello world,2,2,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testDefaultJoinOnTwoCustomTypeInputsWithInnerClassKeyExtractorsClosureCleaner() - throws Exception { - /* - * (Default) Join on two custom type inputs with key extractors, implemented as inner classes to test closure - * cleaning - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); - - DataSet> joinDs = - ds1.join(ds2) - .where( - new KeySelector() { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - }) - .equalTo( - new KeySelector() { - - @Override - public Integer getKey(CustomType value) throws Exception { - return value.myInt; - } - }); - - List> result = joinDs.collect(); - - String expected = - "1,0,Hi,1,0,Hi\n" - + "2,1,Hello,2,1,Hello\n" - + "2,1,Hello,2,2,Hello world\n" - + "2,2,Hello world,2,1,Hello\n" - + "2,2,Hello world,2,2,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void - testDefaultJoinOnTwoCustomTypeInputsWithInnerClassKeyExtractorsDisabledClosureCleaner() - throws Exception { - /* - * (Default) Join on two custom type inputs with key extractors, check if disabling closure cleaning works - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().disableClosureCleaner(); - - DataSet ds1 = CollectionDataSets.getCustomTypeDataSet(env); - DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); - boolean correctExceptionTriggered = false; - try { - DataSet> joinDs = - ds1.join(ds2) - .where( - new KeySelector() { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - }) - .equalTo( - new KeySelector() { - - @Override - public Integer getKey(CustomType value) throws Exception { - return value.myInt; - } - }); - } catch (InvalidProgramException ex) { - correctExceptionTriggered = (ex.getCause() instanceof java.io.NotSerializableException); - } - Assert.assertTrue(correctExceptionTriggered); - } - - private static class KeySelector5 implements KeySelector { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - } - - private static class KeySelector6 implements KeySelector { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - } - - @Test - public void testUDFJoinOnTuplesWithTupleReturningKeySelectors() throws Exception { - /* - * UDF Join on tuples with tuple-returning key selectors - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.get3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - DataSet> joinDs = - ds1.join(ds2) - .where(new KeySelector3()) - .equalTo(new KeySelector4()) - .with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "Hello world,Hallo Welt wie gehts?\n" - + "Hello world,ABC\n" - + "I am fine.,HIJ\n" - + "I am fine.,IJK\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector3 - implements KeySelector, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple3 t) { - return new Tuple2(t.f0, t.f1); - } - } - - private static class KeySelector4 - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 t) { - return new Tuple2(t.f0, t.f4); - } - } - - @Test - public void testJoinNestedPojoAgainstTupleSelectedUsingString() throws Exception { - /* - * Join nested pojo against tuple (selected using a string) - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = ds1.join(ds2).where("nestedPojo.longNumber").equalTo("f6"); - - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinNestedPojoAgainstTupleSelectedUsingInteger() throws Exception { - /* - * Join nested pojo against tuple (selected as an integer) - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.join(ds2).where("nestedPojo.longNumber").equalTo(6); // <--- difference! - - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exception { - /* - * selecting multiple fields using expression language - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.join(ds2) - .where("nestedPojo.longNumber", "number", "str") - .equalTo("f6", "f0", "f1"); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedIntoTuple() throws Exception { - /* - * nested into tuple - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.join(ds2) - .where( - "nestedPojo.longNumber", - "number", - "nestedTupleWithCustom.f0") - .equalTo("f6", "f0", "f2"); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedIntoTupleIntoPojo() throws Exception { - /* - * nested into tuple into pojo - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.join(ds2) - .where( - "nestedTupleWithCustom.f0", - "nestedTupleWithCustom.f1.myInt", - "nestedTupleWithCustom.f1.myLong") - .equalTo("f2", "f3", "f4"); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNonPojoToVerifyFullTupleKeys() throws Exception { - /* - * Non-POJO test to verify that full-tuple keys are working. - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet, String>> ds1 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet, String>> ds2 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - joinDs = - ds1.join(ds2) - .where(0) - .equalTo("f0.f0", "f0.f1"); // key is now Tuple2 - - env.setParallelism(1); - List< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - result = joinDs.collect(); - - String expected = - "((1,1),one),((1,1),one)\n" - + "((2,2),two),((2,2),two)\n" - + "((3,3),three),((3,3),three)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception { - /* - * Non-POJO test to verify "nested" tuple-element selection. - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet, String>> ds1 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet, String>> ds2 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - joinDs = - ds1.join(ds2) - .where("f0.f0") - .equalTo("f0.f0"); // key is now Integer from Tuple2 - - env.setParallelism(1); - List< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - result = joinDs.collect(); - - String expected = - "((1,1),one),((1,1),one)\n" - + "((2,2),two),((2,2),two)\n" - + "((3,3),three),((3,3),three)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testFullPojoWithFullTuple() throws Exception { - /* - * full pojo with full tuple - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env); - DataSet>> - joinDs = ds1.join(ds2).where("*").equalTo("*"); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" - + "2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" - + "3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNonPojoToVerifyNestedTupleElementSelectionWithFirstKeyFieldGreaterThanZero() - throws Exception { - /* - * Non-POJO test to verify "nested" tuple-element selection with the first key field greater than 0. - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet, Tuple3>> ds2 = - ds1.join(ds1).where(0).equalTo(0); - DataSet< - Tuple2< - Tuple2< - Tuple3, - Tuple3>, - Tuple2< - Tuple3, - Tuple3>>> - joinDs = ds2.join(ds2).where("f1.f0").equalTo("f0.f0"); - - env.setParallelism(1); - List< - Tuple2< - Tuple2< - Tuple3, - Tuple3>, - Tuple2< - Tuple3, - Tuple3>>> - result = joinDs.collect(); - - String expected = - "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" - + "((2,2,Hello),(2,2,Hello)),((2,2,Hello),(2,2,Hello))\n" - + "((3,2,Hello world),(3,2,Hello world)),((3,2,Hello world),(3,2,Hello world))\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithAtomicType1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet ds2 = env.fromElements(1, 2); - - DataSet, Integer>> joinDs = - ds1.join(ds2).where(0).equalTo("*"); - - List, Integer>> result = joinDs.collect(); - - String expected = "(1,1,Hi),1\n" + "(2,2,Hello),2"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithAtomicType2() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = env.fromElements(1, 2); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - - DataSet>> joinDs = - ds1.join(ds2).where("*").equalTo(0); - - List>> result = joinDs.collect(); - - String expected = "1,(1,1,Hi)\n" + "2,(2,2,Hello)"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithRangePartitioning() throws Exception { - /* - * Test Join on tuples with multiple key field positions and same customized distribution - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.get3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.get5TupleDataSet(env); - - env.setParallelism(4); - TestDistribution testDis = new TestDistribution(); - DataSet> joinDs = - DataSetUtils.partitionByRange(ds1, testDis, 0, 1) - .join(DataSetUtils.partitionByRange(ds2, testDis, 0, 4)) - .where(0, 1) - .equalTo(0, 4) - .with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "Hello world,Hallo Welt wie gehts?\n" - + "Hello world,ABC\n" - + "I am fine.,HIJ\n" - + "I am fine.,IJK\n"; - - compareResultAsTuples(result, expected); - } - - private static class T3T5FlatJoin - implements FlatJoinFunction< - Tuple3, - Tuple5, - Tuple2> { - - @Override - public void join( - Tuple3 first, - Tuple5 second, - Collector> out) { - - out.collect(new Tuple2(first.f2, second.f3)); - } - } - - private static class LeftReturningJoin - implements JoinFunction< - Tuple3, - Tuple5, - Tuple3> { - - @Override - public Tuple3 join( - Tuple3 first, - Tuple5 second) { - - return first; - } - } - - private static class RightReturningJoin - implements JoinFunction< - Tuple3, - Tuple5, - Tuple5> { - - @Override - public Tuple5 join( - Tuple3 first, - Tuple5 second) { - - return second; - } - } - - private static class T3T5BCJoin - extends RichFlatJoinFunction< - Tuple3, - Tuple5, - Tuple3> { - - private int broadcast; - - @Override - public void open(OpenContext openContext) { - - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - broadcast = sum; - } - - /* - @Override - public Tuple3 join( - Tuple3 first, - Tuple5 second) { - - return new Tuple3(first.f2, second.f3, broadcast); - } - */ - - @Override - public void join( - Tuple3 first, - Tuple5 second, - Collector> out) - throws Exception { - out.collect(new Tuple3(first.f2, second.f3, broadcast)); - } - } - - private static class T3CustJoin - implements JoinFunction< - Tuple3, CustomType, Tuple2> { - - @Override - public Tuple2 join(Tuple3 first, CustomType second) { - - return new Tuple2(first.f2, second.myString); - } - } - - private static class CustT3Join - implements JoinFunction< - CustomType, Tuple3, Tuple2> { - - @Override - public Tuple2 join(CustomType first, Tuple3 second) { - - return new Tuple2(first.myString, second.f2); - } - } - - /** Test data distribution. */ - public static class TestDistribution implements DataDistribution { - public Object[][] boundaries = - new Object[][] { - new Object[] {2, 2L}, - new Object[] {5, 4L}, - new Object[] {10, 12L}, - new Object[] {21, 6L} - }; - - public TestDistribution() {} - - @Override - public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) { - return boundaries[bucketNum]; - } - - @Override - public int getNumberOfFields() { - return 2; - } - - @Override - public TypeInformation[] getKeyTypes() { - return new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO - }; - } - - @Override - public void write(DataOutputView out) throws IOException {} - - @Override - public void read(DataInputView in) throws IOException {} - - @Override - public boolean equals(Object obj) { - return obj instanceof TestDistribution; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java deleted file mode 100644 index ecb3bed4702b1..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java +++ /dev/null @@ -1,566 +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.operators; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.configuration.ConfigurationUtils.getIntConfigOption; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link MapFunction} and {@link RichMapFunction}. */ -@RunWith(Parameterized.class) -public class MapITCase extends MultipleProgramsTestBaseJUnit4 { - - public MapITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testIdentityMapWithBasicType() throws Exception { - /* - * Test identity map with basic type - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getStringDataSet(env); - DataSet identityMapDs = ds.map(new Mapper1()); - - List result = identityMapDs.collect(); - - String expected = - "Hi\n" - + "Hello\n" - + "Hello world\n" - + "Hello world, how are you?\n" - + "I am fine.\n" - + "Luke Skywalker\n" - + "Random comment\n" - + "LOL\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testRuntimeContextAndExecutionConfigParams() throws Exception { - /* - * Test identity map with basic type - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().enableObjectReuse(); - String key = "key"; - String value = "value"; - Configuration configuration = Configuration.fromMap(Collections.singletonMap(key, value)); - env.getConfig().setGlobalJobParameters(configuration); - - DataSet ds = CollectionDataSets.getStringDataSet(env); - DataSet identityMapDs = - ds.map( - new RichMapFunction() { - @Override - public String map(String value) throws Exception { - Assert.assertTrue(getRuntimeContext().isObjectReuseEnabled()); - Assert.assertEquals( - getRuntimeContext().getGlobalJobParameters(), - configuration.toMap()); - return value; - } - }); - - List result = identityMapDs.collect(); - - String expected = - "Hi\n" - + "Hello\n" - + "Hello world\n" - + "Hello world, how are you?\n" - + "I am fine.\n" - + "Luke Skywalker\n" - + "Random comment\n" - + "LOL\n"; - - compareResultAsText(result, expected); - } - - private static class Mapper1 implements MapFunction { - private static final long serialVersionUID = 1L; - - @Override - public String map(String value) throws Exception { - return value; - } - } - - @Test - public void testIdentityMapWithTuple() throws Exception { - /* - * Test identity map with a tuple - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> identityMapDs = ds.map(new Mapper2()); - - List> result = identityMapDs.collect(); - - String expected = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "6,3,Luke Skywalker\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "9,4,Comment#3\n" - + "10,4,Comment#4\n" - + "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n" - + "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "18,6,Comment#12\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "21,6,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class Mapper2 - implements MapFunction, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - return value; - } - } - - @Test - public void testTypeConversionMapperCustomToTuple() throws Exception { - /* - * Test type conversion mapper (Custom -> Tuple) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet> typeConversionMapDs = ds.map(new Mapper3()); - - List> result = typeConversionMapDs.collect(); - - String expected = - "1,0,Hi\n" - + "2,1,Hello\n" - + "2,2,Hello world\n" - + "3,3,Hello world, how are you?\n" - + "3,4,I am fine.\n" - + "3,5,Luke Skywalker\n" - + "4,6,Comment#1\n" - + "4,7,Comment#2\n" - + "4,8,Comment#3\n" - + "4,9,Comment#4\n" - + "5,10,Comment#5\n" - + "5,11,Comment#6\n" - + "5,12,Comment#7\n" - + "5,13,Comment#8\n" - + "5,14,Comment#9\n" - + "6,15,Comment#10\n" - + "6,16,Comment#11\n" - + "6,17,Comment#12\n" - + "6,18,Comment#13\n" - + "6,19,Comment#14\n" - + "6,20,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class Mapper3 implements MapFunction> { - private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); - - @Override - public Tuple3 map(CustomType value) throws Exception { - out.setField(value.myInt, 0); - out.setField(value.myLong, 1); - out.setField(value.myString, 2); - return out; - } - } - - @Test - public void testTypeConversionMapperTupleToBasic() throws Exception { - /* - * Test type conversion mapper (Tuple -> Basic) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet typeConversionMapDs = ds.map(new Mapper4()); - - List result = typeConversionMapDs.collect(); - - String expected = - "Hi\n" - + "Hello\n" - + "Hello world\n" - + "Hello world, how are you?\n" - + "I am fine.\n" - + "Luke Skywalker\n" - + "Comment#1\n" - + "Comment#2\n" - + "Comment#3\n" - + "Comment#4\n" - + "Comment#5\n" - + "Comment#6\n" - + "Comment#7\n" - + "Comment#8\n" - + "Comment#9\n" - + "Comment#10\n" - + "Comment#11\n" - + "Comment#12\n" - + "Comment#13\n" - + "Comment#14\n" - + "Comment#15\n"; - - compareResultAsText(result, expected); - } - - private static class Mapper4 implements MapFunction, String> { - private static final long serialVersionUID = 1L; - - @Override - public String map(Tuple3 value) throws Exception { - return value.getField(2); - } - } - - @Test - public void testMapperOnTupleIncrementIntegerFieldReorderSecondAndThirdFields() - throws Exception { - /* - * Test mapper on tuple - Increment Integer field, reorder second and third fields - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> tupleMapDs = ds.map(new Mapper5()); - - List> result = tupleMapDs.collect(); - - String expected = - "2,Hi,1\n" - + "3,Hello,2\n" - + "4,Hello world,2\n" - + "5,Hello world, how are you?,3\n" - + "6,I am fine.,3\n" - + "7,Luke Skywalker,3\n" - + "8,Comment#1,4\n" - + "9,Comment#2,4\n" - + "10,Comment#3,4\n" - + "11,Comment#4,4\n" - + "12,Comment#5,5\n" - + "13,Comment#6,5\n" - + "14,Comment#7,5\n" - + "15,Comment#8,5\n" - + "16,Comment#9,5\n" - + "17,Comment#10,6\n" - + "18,Comment#11,6\n" - + "19,Comment#12,6\n" - + "20,Comment#13,6\n" - + "21,Comment#14,6\n" - + "22,Comment#15,6\n"; - - compareResultAsTuples(result, expected); - } - - private static class Mapper5 - implements MapFunction, Tuple3> { - private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); - - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - Integer incr = Integer.valueOf(value.f0.intValue() + 1); - out.setFields(incr, value.f2, value.f1); - return out; - } - } - - @Test - public void testMapperOnCustomLowercaseString() throws Exception { - /* - * Test mapper on Custom - lowercase myString - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet customMapDs = ds.map(new Mapper6()); - - List result = customMapDs.collect(); - - String expected = - "1,0,hi\n" - + "2,1,hello\n" - + "2,2,hello world\n" - + "3,3,hello world, how are you?\n" - + "3,4,i am fine.\n" - + "3,5,luke skywalker\n" - + "4,6,comment#1\n" - + "4,7,comment#2\n" - + "4,8,comment#3\n" - + "4,9,comment#4\n" - + "5,10,comment#5\n" - + "5,11,comment#6\n" - + "5,12,comment#7\n" - + "5,13,comment#8\n" - + "5,14,comment#9\n" - + "6,15,comment#10\n" - + "6,16,comment#11\n" - + "6,17,comment#12\n" - + "6,18,comment#13\n" - + "6,19,comment#14\n" - + "6,20,comment#15\n"; - - compareResultAsText(result, expected); - } - - private static class Mapper6 implements MapFunction { - private static final long serialVersionUID = 1L; - private final CustomType out = new CustomType(); - - @Override - public CustomType map(CustomType value) throws Exception { - out.myInt = value.myInt; - out.myLong = value.myLong; - out.myString = value.myString.toLowerCase(); - return out; - } - } - - @Test - public void test() throws Exception { - /* - * Test mapper if UDF returns input object - increment first field of a tuple - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> inputObjMapDs = ds.map(new Mapper7()); - - List> result = inputObjMapDs.collect(); - - String expected = - "2,1,Hi\n" - + "3,2,Hello\n" - + "4,2,Hello world\n" - + "5,3,Hello world, how are you?\n" - + "6,3,I am fine.\n" - + "7,3,Luke Skywalker\n" - + "8,4,Comment#1\n" - + "9,4,Comment#2\n" - + "10,4,Comment#3\n" - + "11,4,Comment#4\n" - + "12,5,Comment#5\n" - + "13,5,Comment#6\n" - + "14,5,Comment#7\n" - + "15,5,Comment#8\n" - + "16,5,Comment#9\n" - + "17,6,Comment#10\n" - + "18,6,Comment#11\n" - + "19,6,Comment#12\n" - + "20,6,Comment#13\n" - + "21,6,Comment#14\n" - + "22,6,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class Mapper7 - implements MapFunction, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - Integer incr = Integer.valueOf(value.f0.intValue() + 1); - value.setField(incr, 0); - return value; - } - } - - @Test - public void testMapWithBroadcastSet() throws Exception { - /* - * Test map with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ints = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> bcMapDs = - ds.map(new RichMapper1()).withBroadcastSet(ints, "ints"); - List> result = bcMapDs.collect(); - - String expected = - "55,1,Hi\n" - + "55,2,Hello\n" - + "55,2,Hello world\n" - + "55,3,Hello world, how are you?\n" - + "55,3,I am fine.\n" - + "55,3,Luke Skywalker\n" - + "55,4,Comment#1\n" - + "55,4,Comment#2\n" - + "55,4,Comment#3\n" - + "55,4,Comment#4\n" - + "55,5,Comment#5\n" - + "55,5,Comment#6\n" - + "55,5,Comment#7\n" - + "55,5,Comment#8\n" - + "55,5,Comment#9\n" - + "55,6,Comment#10\n" - + "55,6,Comment#11\n" - + "55,6,Comment#12\n" - + "55,6,Comment#13\n" - + "55,6,Comment#14\n" - + "55,6,Comment#15\n"; - - compareResultAsTuples(result, expected); - } - - private static class RichMapper1 - extends RichMapFunction, Tuple3> { - private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); - private Integer f2Replace = 0; - - @Override - public void open(OpenContext openContext) { - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - f2Replace = sum; - } - - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - out.setFields(f2Replace, value.f1, value.f2); - return out; - } - } - - static final String TEST_KEY = "testVariable"; - static final int TEST_VALUE = 666; - - @Test - public void testPassingConfigurationObject() throws Exception { - /* - * Test passing configuration object. - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - Configuration conf = new Configuration(); - conf.set(getIntConfigOption(TEST_KEY), TEST_VALUE); - DataSet> bcMapDs = - ds.map(new RichMapper2()).withParameters(conf); - List> result = bcMapDs.collect(); - - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world"; - - compareResultAsTuples(result, expected); - } - - private static class RichMapper2 - extends RichMapFunction, Tuple3> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 map(Tuple3 value) { - return value; - } - } - - @Test - public void testMapWithLambdas() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet stringDs = env.fromElements(11, 12, 13, 14); - DataSet mappedDs = - stringDs.map(Object::toString) - .map(s -> s.replace("1", "2")) - .map(Trade::new) - .map(Trade::toString); - List result = mappedDs.collect(); - - String expected = "22\n" + "22\n" + "23\n" + "24\n"; - - compareResultAsText(result, expected); - } - - private static class Trade { - - public String v; - - public Trade(String v) { - this.v = v; - } - - @Override - public String toString() { - return v; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java deleted file mode 100644 index 44ee39249a799..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java +++ /dev/null @@ -1,106 +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.operators; - -import org.apache.flink.api.common.functions.MapPartitionFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.util.Collector; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultCollections; - -/** Integration tests for {@link MapPartitionFunction}. */ -@SuppressWarnings("serial") -public class MapPartitionITCase extends JavaProgramTestBaseJUnit4 { - - private static final String IN = - "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n" - + "1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n1 1\n2 2\n2 2\n3 0\n4 4\n" - + "5 9\n7 7\n8 8\n1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n"; - - private static final String RESULT = - "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n"; - - private List> input = new ArrayList<>(); - - private List> expected = new ArrayList<>(); - - private List> result = new ArrayList<>(); - - @Override - protected void preSubmit() throws Exception { - - // create input - for (String s : IN.split("\n")) { - String[] fields = s.split(" "); - input.add(new Tuple2(fields[0], fields[1])); - } - - // create expected - for (String s : RESULT.split("\n")) { - String[] fields = s.split(" "); - expected.add(new Tuple2(fields[0], Integer.parseInt(fields[1]))); - } - } - - @Override - protected void postSubmit() { - compareResultCollections( - expected, result, new TestBaseUtils.TupleComparator>()); - } - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = env.fromCollection(input); - - data.mapPartition(new TestMapPartition()) - .output(new LocalCollectionOutputFormat>(result)); - - env.execute(); - } - - private static class TestMapPartition - implements MapPartitionFunction, Tuple2> { - - @Override - public void mapPartition( - Iterable> values, Collector> out) { - for (Tuple2 value : values) { - String keyString = value.f0; - String valueString = value.f1; - - int keyInt = Integer.parseInt(keyString); - int valueInt = Integer.parseInt(valueString); - - if (keyInt + valueInt < 10) { - out.collect(new Tuple2(valueString, keyInt + 10)); - } - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java index 029a10cfd0811..37c291837fd55 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java @@ -18,23 +18,21 @@ package org.apache.flink.test.operators; -import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.ReduceFunction; -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.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.test.util.TestBaseUtils.TupleComparator; -import org.apache.flink.util.Collector; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.List; import static org.junit.Assert.assertEquals; @@ -69,18 +67,18 @@ public ObjectReuseITCase(boolean objectReuse) { @Test public void testKeyedReduce() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); if (objectReuse) { env.getConfig().enableObjectReuse(); } else { env.getConfig().disableObjectReuse(); } - DataSet> input = env.fromCollection(REDUCE_DATA); + DataStreamSource> input = env.fromData(REDUCE_DATA); - DataSet> result = - input.groupBy(0) + DataStream> result = + input.keyBy(x -> x.f0) .reduce( new ReduceFunction>() { @@ -93,150 +91,45 @@ public Tuple2 reduce( } }); - Tuple2 res = result.collect().get(0); + Tuple2 res = result.executeAndCollect().next(); assertEquals(new Tuple2<>("a", 60), res); } @Test public void testGlobalReduce() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); if (objectReuse) { env.getConfig().enableObjectReuse(); } else { env.getConfig().disableObjectReuse(); } - DataSet> input = env.fromCollection(REDUCE_DATA); - - DataSet> result = - input.reduce( - new ReduceFunction>() { - - @Override - public Tuple2 reduce( - Tuple2 value1, - Tuple2 value2) { - - if (value1.f1 % 3 == 0) { - value1.f1 += value2.f1; - return value1; - } else { - value2.f1 += value1.f1; - return value2; - } - } - }); - - Tuple2 res = result.collect().get(0); - assertEquals(new Tuple2<>("a", 60), res); - } - - @Test - public void testKeyedGroupReduce() throws Exception { + DataStreamSource> input = env.fromData(REDUCE_DATA); - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - if (objectReuse) { - env.getConfig().enableObjectReuse(); - } else { - env.getConfig().disableObjectReuse(); - } - - DataSet> input = env.fromCollection(GROUP_REDUCE_DATA); - - DataSet> result = - input.groupBy(0) - .reduceGroup( - new GroupReduceFunction< - Tuple2, Tuple2>() { + DataStream> result = + input.windowAll(GlobalWindows.createWithEndOfStreamTrigger()) + .reduce( + new ReduceFunction>() { @Override - public void reduce( - Iterable> values, - Collector> out) { - List> list = new ArrayList<>(); - for (Tuple2 val : values) { - list.add(val); - } + public Tuple2 reduce( + Tuple2 value1, + Tuple2 value2) { - for (Tuple2 val : list) { - out.collect(val); + if (value1.f1 % 3 == 0) { + value1.f1 += value2.f1; + return value1; + } else { + value2.f1 += value1.f1; + return value2; } } }); - List> is = result.collect(); - Collections.sort(is, new TupleComparator>()); - - List> expected = - env.getConfig().isObjectReuseEnabled() - ? Arrays.asList( - new Tuple2<>("a", 4), - new Tuple2<>("a", 4), - new Tuple2<>("a", 5), - new Tuple2<>("a", 5), - new Tuple2<>("a", 5)) - : Arrays.asList( - new Tuple2<>("a", 1), - new Tuple2<>("a", 2), - new Tuple2<>("a", 3), - new Tuple2<>("a", 4), - new Tuple2<>("a", 5)); - - assertEquals(expected, is); - } - - @Test - public void testGlobalGroupReduce() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - if (objectReuse) { - env.getConfig().enableObjectReuse(); - } else { - env.getConfig().disableObjectReuse(); - } - - DataSet> input = env.fromCollection(GROUP_REDUCE_DATA); - - DataSet> result = - input.reduceGroup( - new GroupReduceFunction< - Tuple2, Tuple2>() { - - @Override - public void reduce( - Iterable> values, - Collector> out) { - List> list = new ArrayList<>(); - for (Tuple2 val : values) { - list.add(val); - } - - for (Tuple2 val : list) { - out.collect(val); - } - } - }); - - List> is = result.collect(); - Collections.sort(is, new TupleComparator>()); - - List> expected = - env.getConfig().isObjectReuseEnabled() - ? Arrays.asList( - new Tuple2<>("a", 4), - new Tuple2<>("a", 4), - new Tuple2<>("a", 5), - new Tuple2<>("a", 5), - new Tuple2<>("a", 5)) - : Arrays.asList( - new Tuple2<>("a", 1), - new Tuple2<>("a", 2), - new Tuple2<>("a", 3), - new Tuple2<>("a", 4), - new Tuple2<>("a", 5)); - - assertEquals(expected, is); + Tuple2 res = result.executeAndCollect().next(); + assertEquals(new Tuple2<>("a", 60), res); } @Parameterized.Parameters(name = "Execution mode = CLUSTER, Reuse = {0}") diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java deleted file mode 100644 index 8777b4a94a54d..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java +++ /dev/null @@ -1,811 +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.operators; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.JoinFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichFlatJoinFunction; -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** - * Integration tests for {@link JoinFunction}, {@link FlatJoinFunction}, and {@link - * RichFlatJoinFunction}. - */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class OuterJoinITCase extends MultipleProgramsTestBaseJUnit4 { - - public OuterJoinITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testLeftOuterJoin1() throws Exception { - testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE); - } - - @Test - public void testLeftOuterJoin2() throws Exception { - testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST); - } - - @Test - public void testLeftOuterJoin3() throws Exception { - testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND); - } - - @Test - public void testLeftOuterJoin4() throws Exception { - testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND); - } - - @Test(expected = InvalidProgramException.class) - public void testLeftOuterJoin5() throws Exception { - testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST); - } - - private void testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception { - /* - * UDF Join on tuples with key field positions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.leftOuterJoin(ds2, hint).where(0).equalTo(0).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "Hello,Hallo Welt wie\n" - + "Hello world,null\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testRightOuterJoin1() throws Exception { - testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE); - } - - @Test - public void testRightOuterJoin2() throws Exception { - testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST); - } - - @Test - public void testRightOuterJoin3() throws Exception { - testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND); - } - - @Test - public void testRightOuterJoin4() throws Exception { - testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST); - } - - @Test(expected = InvalidProgramException.class) - public void testRightOuterJoin5() throws Exception { - testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND); - } - - private void testRightOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception { - /* - * UDF Join on tuples with key field positions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.rightOuterJoin(ds2, hint).where(1).equalTo(1).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "null,Hallo Welt wie\n" - + "Hello world,Hallo Welt\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testFullOuterJoin1() throws Exception { - testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE); - } - - @Test - public void testFullOuterJoin2() throws Exception { - testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST); - } - - @Test - public void testFullOuterJoin3() throws Exception { - testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND); - } - - @Test(expected = InvalidProgramException.class) - public void testFullOuterJoin4() throws Exception { - testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST); - } - - @Test(expected = InvalidProgramException.class) - public void testFullOuterJoin5() throws Exception { - testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND); - } - - private void testFullOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception { - /* - * UDF Join on tuples with key field positions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.fullOuterJoin(ds2, hint).where(0).equalTo(2).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "null,Hallo\n" - + "Hi,Hallo Welt\n" - + "Hello,Hallo Welt wie\n" - + "Hello world,null\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinOnTuplesWithCompositeKeyPositions() throws Exception { - /* - * UDF Join on tuples with multiple key field positions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.fullOuterJoin(ds2).where(0, 1).equalTo(0, 4).with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "Hello world,null\n" - + "null,Hallo Welt wie\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithBroadcastSet() throws Exception { - /* - * Join with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.fullOuterJoin(ds2) - .where(1) - .equalTo(4) - .with(new T3T5BCJoin()) - .withBroadcastSet(intDs, "ints"); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo,55\n" - + "Hi,Hallo Welt wie,55\n" - + "Hello,Hallo Welt,55\n" - + "Hello world,Hallo Welt,55\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithMixedKeyTypes1() throws Exception { - /* - * Join on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> joinDs = - ds1.fullOuterJoin(ds2).where(new KeySelector1()).equalTo(0).with(new CustT3Join()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hi\n" + "Hello,Hello\n" + "Hello world,Hello\n" + "null,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector1 implements KeySelector { - @Override - public Integer getKey(CustomType value) { - return value.myInt; - } - } - - @Test - public void testJoinWithMixedKeyTypes2() throws Exception { - /* - * Join on a tuple input with key field selector and a custom type input with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env); - DataSet> joinDs = - ds1.fullOuterJoin(ds2).where(1).equalTo(new KeySelector2()).with(new T3CustJoin()); - - List> result = joinDs.collect(); - - String expected = - "null,Hi\n" + "Hi,Hello\n" + "Hello,Hello world\n" + "Hello world,Hello world\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector2 implements KeySelector { - @Override - public Long getKey(CustomType value) { - return value.myLong; - } - } - - @Test - public void testJoinWithTupleReturningKeySelectors() throws Exception { - /* - * UDF Join on tuples with tuple-returning key selectors - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmall5TupleDataSet(env); - DataSet> joinDs = - ds1.fullOuterJoin(ds2) - .where(new KeySelector3()) // 0, 1 - .equalTo(new KeySelector4()) // 0, 4 - .with(new T3T5FlatJoin()); - - List> result = joinDs.collect(); - - String expected = - "Hi,Hallo\n" - + "Hello,Hallo Welt\n" - + "Hello world,null\n" - + "null,Hallo Welt wie\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector3 - implements KeySelector, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple3 t) { - return new Tuple2<>(t.f0, t.f1); - } - } - - private static class KeySelector4 - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 t) { - return new Tuple2<>(t.f0, t.f4); - } - } - - @Test - public void testJoinWithNestedKeyExpression1() throws Exception { - /* - * Join nested pojo against tuple (selected using a string) - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.fullOuterJoin(ds2) - .where("nestedPojo.longNumber") - .equalTo("f6") - .with( - new ProjectBothFunction< - POJO, - Tuple7< - Integer, - String, - Integer, - Integer, - Long, - String, - Long>>()); - - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithNestedKeyExpression2() throws Exception { - /* - * Join nested pojo against tuple (selected as an integer) - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.fullOuterJoin(ds2) - .where("nestedPojo.longNumber") - .equalTo(6) // <--- difference! - .with( - new ProjectBothFunction< - POJO, - Tuple7< - Integer, - String, - Integer, - Integer, - Long, - String, - Long>>()); - - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithCompositeKeyExpressions() throws Exception { - /* - * selecting multiple fields using expression language - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.fullOuterJoin(ds2) - .where("nestedPojo.longNumber", "number", "str") - .equalTo("f6", "f0", "f1") - .with( - new ProjectBothFunction< - POJO, - Tuple7< - Integer, - String, - Integer, - Integer, - Long, - String, - Long>>()); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedIntoTuple() throws Exception { - /* - * nested into tuple - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.fullOuterJoin(ds2) - .where( - "nestedPojo.longNumber", - "number", - "nestedTupleWithCustom.f0") - .equalTo("f6", "f0", "f2") - .with( - new ProjectBothFunction< - POJO, - Tuple7< - Integer, - String, - Integer, - Integer, - Long, - String, - Long>>()); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedIntoTupleIntoPojo() throws Exception { - /* - * nested into tuple into pojo - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSet(env); - DataSet>> - joinDs = - ds1.fullOuterJoin(ds2) - .where( - "nestedTupleWithCustom.f0", - "nestedTupleWithCustom.f1.myInt", - "nestedTupleWithCustom.f1.myLong") - .equalTo("f2", "f3", "f4") - .with( - new ProjectBothFunction< - POJO, - Tuple7< - Integer, - String, - Integer, - Integer, - Long, - String, - Long>>()); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" - + "2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" - + "3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNonPojoToVerifyFullTupleKeys() throws Exception { - /* - * Non-POJO test to verify that full-tuple keys are working. - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet, String>> ds1 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet, String>> ds2 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - joinDs = - ds1.fullOuterJoin(ds2) - .where(0) - .equalTo("f0.f0", "f0.f1") // key is now Tuple2 - .with( - new ProjectBothFunction< - Tuple2, String>, - Tuple2, String>>()); - - env.setParallelism(1); - List< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - result = joinDs.collect(); - - String expected = - "((1,1),one),((1,1),one)\n" - + "((2,2),two),((2,2),two)\n" - + "((3,3),three),((3,3),three)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception { - /* - * Non-POJO test to verify "nested" tuple-element selection. - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet, String>> ds1 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet, String>> ds2 = - CollectionDataSets.getSmallNestedTupleDataSet(env); - DataSet< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - joinDs = - ds1.fullOuterJoin(ds2) - .where("f0.f0") - .equalTo( - "f0.f0") // key is now Integer from Tuple2 - .with( - new ProjectBothFunction< - Tuple2, String>, - Tuple2, String>>()); - - env.setParallelism(1); - List< - Tuple2< - Tuple2, String>, - Tuple2, String>>> - result = joinDs.collect(); - - String expected = - "((1,1),one),((1,1),one)\n" - + "((2,2),two),((2,2),two)\n" - + "((3,3),three),((3,3),three)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testFullPojoWithFullTuple() throws Exception { - /* - * full pojo with full tuple - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = CollectionDataSets.getSmallPojoDataSet(env); - DataSet> ds2 = - CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env); - DataSet>> - joinDs = - ds1.fullOuterJoin(ds2) - .where("*") - .equalTo("*") - .with( - new ProjectBothFunction< - POJO, - Tuple7< - Long, - Integer, - Integer, - Long, - String, - Integer, - String>>()); - - env.setParallelism(1); - List>> result = - joinDs.collect(); - - String expected = - "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" - + "2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" - + "3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithAtomicType1() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet ds2 = env.fromElements(1, 2); - - DataSet, Integer>> joinDs = - ds1.fullOuterJoin(ds2) - .where(0) - .equalTo("*") - .with(new ProjectBothFunction, Integer>()) - .returns(new GenericTypeInfo(Tuple2.class)); - - List, Integer>> result = joinDs.collect(); - - String expected = "(1,1,Hi),1\n" + "(2,2,Hello),2\n" + "(3,2,Hello world),null\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testJoinWithAtomicType2() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds1 = env.fromElements(1, 2); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - - DataSet>> joinDs = - ds1.fullOuterJoin(ds2) - .where("*") - .equalTo(0) - .with(new ProjectBothFunction>()) - .returns(new GenericTypeInfo(Tuple2.class)); - - List>> result = joinDs.collect(); - - String expected = "1,(1,1,Hi)\n" + "2,(2,2,Hello)\n" + "null,(3,2,Hello world)\n"; - - compareResultAsTuples(result, expected); - } - - private static class T3T5FlatJoin - implements FlatJoinFunction< - Tuple3, - Tuple5, - Tuple2> { - - @Override - public void join( - Tuple3 first, - Tuple5 second, - Collector> out) { - - out.collect( - new Tuple2<>( - first == null ? null : first.f2, second == null ? null : second.f3)); - } - } - - private static class T3T5BCJoin - extends RichFlatJoinFunction< - Tuple3, - Tuple5, - Tuple3> { - - private int broadcast; - - @Override - public void open(OpenContext openContext) { - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - broadcast = sum; - } - - @Override - public void join( - Tuple3 first, - Tuple5 second, - Collector> out) - throws Exception { - out.collect( - new Tuple3<>( - first == null ? null : first.f2, - second == null ? null : second.f3, - broadcast)); - } - } - - private static class T3CustJoin - implements JoinFunction< - Tuple3, CustomType, Tuple2> { - - @Override - public Tuple2 join(Tuple3 first, CustomType second) { - - return new Tuple2<>( - first == null ? null : first.f2, second == null ? null : second.myString); - } - } - - private static class CustT3Join - implements JoinFunction< - CustomType, Tuple3, Tuple2> { - - @Override - public Tuple2 join(CustomType first, Tuple3 second) { - - return new Tuple2<>( - first == null ? null : first.myString, second == null ? null : second.f2); - } - } - - /** Deliberately untyped join function, which emits a Tuple2 of the left and right side. */ - private static class ProjectBothFunction - implements JoinFunction> { - @Override - public Tuple2 join(IN1 first, IN2 second) throws Exception { - return new Tuple2<>(first, second); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java deleted file mode 100644 index 173688cabd42a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java +++ /dev/null @@ -1,850 +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.operators; - -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.MapPartitionFunction; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.operators.Order; -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.AggregateOperator; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.Serializable; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** Integration tests for {@link MapPartitionFunction}. */ -@RunWith(Parameterized.class) -@SuppressWarnings("serial") -public class PartitionITCase extends MultipleProgramsTestBaseJUnit4 { - - public PartitionITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testHashPartitionByKeyField() throws Exception { - /* - * Test hash partition by key field - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet uniqLongs = ds.partitionByHash(1).mapPartition(new UniqueTupleLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testRangePartitionByKeyField() throws Exception { - /* - * Test range partition by key field - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet uniqLongs = ds.partitionByRange(1).mapPartition(new UniqueTupleLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testHashPartitionByKeyField2() throws Exception { - /* - * Test hash partition by key field - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - AggregateOperator> sum = - ds.map(new PrefixMapper()).partitionByHash(1, 2).groupBy(1, 2).sum(0); - - List> result = sum.collect(); - - String expected = - "(1,1,Hi)\n" - + "(5,2,Hello)\n" - + "(4,3,Hello)\n" - + "(5,3,I am )\n" - + "(6,3,Luke )\n" - + "(34,4,Comme)\n" - + "(65,5,Comme)\n" - + "(111,6,Comme)"; - - compareResultAsText(result, expected); - } - - @Test - public void testRangePartitionByKeyField2() throws Exception { - /* - * Test range partition by key field - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - AggregateOperator> sum = - ds.map(new PrefixMapper()).partitionByRange(1, 2).groupBy(1, 2).sum(0); - - List> result = sum.collect(); - - String expected = - "(1,1,Hi)\n" - + "(5,2,Hello)\n" - + "(4,3,Hello)\n" - + "(5,3,I am )\n" - + "(6,3,Luke )\n" - + "(34,4,Comme)\n" - + "(65,5,Comme)\n" - + "(111,6,Comme)"; - - compareResultAsText(result, expected); - } - - @Test - public void testHashPartitionOfAtomicType() throws Exception { - /* - * Test hash partition of atomic type - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet uniqLongs = - env.generateSequence(1, 6) - .union(env.generateSequence(1, 6)) - .rebalance() - .partitionByHash("*") - .mapPartition(new UniqueLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testRangePartitionOfAtomicType() throws Exception { - /* - * Test range partition of atomic type - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet uniqLongs = - env.generateSequence(1, 6) - .union(env.generateSequence(1, 6)) - .rebalance() - .partitionByRange("*") - .mapPartition(new UniqueLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testHashPartitionByKeySelector() throws Exception { - /* - * Test hash partition by key selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet uniqLongs = - ds.partitionByHash(new KeySelector1()).mapPartition(new UniqueTupleLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - private static class PrefixMapper - implements MapFunction, Tuple3> { - @Override - public Tuple3 map(Tuple3 value) - throws Exception { - if (value.f2.length() > 5) { - value.f2 = value.f2.substring(0, 5); - } - return value; - } - } - - @Test - public void testRangePartitionByKeySelector() throws Exception { - /* - * Test range partition by key selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet uniqLongs = - ds.partitionByRange(new KeySelector1()).mapPartition(new UniqueTupleLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector1 implements KeySelector, Long> { - private static final long serialVersionUID = 1L; - - @Override - public Long getKey(Tuple3 value) throws Exception { - return value.f1; - } - } - - @Test - public void testForcedRebalancing() throws Exception { - /* - * Test forced rebalancing - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // generate some number in parallel - DataSet ds = env.generateSequence(1, 3000); - DataSet> uniqLongs = - ds - // introduce some partition skew by filtering - .filter(new Filter1()) - // rebalance - .rebalance() - // count values in each partition - .map(new PartitionIndexMapper()) - .groupBy(0) - .reduce(new Reducer1()) - // round counts to mitigate runtime scheduling effects (lazy split - // assignment) - .map(new Mapper1()); - - List> result = uniqLongs.collect(); - - StringBuilder expected = new StringBuilder(); - int numPerPartition = 2220 / env.getParallelism() / 10; - for (int i = 0; i < env.getParallelism(); i++) { - expected.append('(').append(i).append(',').append(numPerPartition).append(")\n"); - } - - compareResultAsText(result, expected.toString()); - } - - private static class Filter1 implements FilterFunction { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Long value) throws Exception { - return value > 780; - } - } - - private static class Reducer1 implements ReduceFunction> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 reduce( - Tuple2 v1, Tuple2 v2) { - return new Tuple2<>(v1.f0, v1.f1 + v2.f1); - } - } - - private static class Mapper1 - implements MapFunction, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - value.f1 = (value.f1 / 10); - return value; - } - } - - @Test - public void testHashPartitionByKeyFieldAndDifferentParallelism() throws Exception { - /* - * Test hash partition by key field and different parallelism - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet uniqLongs = - ds.partitionByHash(1).setParallelism(4).mapPartition(new UniqueTupleLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testRangePartitionByKeyFieldAndDifferentParallelism() throws Exception { - /* - * Test range partition by key field and different parallelism - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet uniqLongs = - ds.partitionByRange(1).setParallelism(4).mapPartition(new UniqueTupleLongMapper()); - List result = uniqLongs.collect(); - - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testHashPartitionWithKeyExpression() throws Exception { - /* - * Test hash partition with key expression - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); - DataSet uniqLongs = - ds.partitionByHash("nestedPojo.longNumber") - .setParallelism(4) - .mapPartition(new UniqueNestedPojoLongMapper()); - List result = uniqLongs.collect(); - - String expected = "10000\n" + "20000\n" + "30000\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testRangePartitionWithKeyExpression() throws Exception { - /* - * Test range partition with key expression - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet ds = CollectionDataSets.getDuplicatePojoDataSet(env); - DataSet uniqLongs = - ds.partitionByRange("nestedPojo.longNumber") - .setParallelism(4) - .mapPartition(new UniqueNestedPojoLongMapper()); - List result = uniqLongs.collect(); - - String expected = "10000\n" + "20000\n" + "30000\n"; - - compareResultAsText(result, expected); - } - - private static class UniqueTupleLongMapper - implements MapPartitionFunction, Long> { - private static final long serialVersionUID = 1L; - - @Override - public void mapPartition( - Iterable> records, Collector out) - throws Exception { - HashSet uniq = new HashSet<>(); - for (Tuple3 t : records) { - uniq.add(t.f1); - } - for (Long l : uniq) { - out.collect(l); - } - } - } - - private static class UniqueLongMapper implements MapPartitionFunction { - private static final long serialVersionUID = 1L; - - @Override - public void mapPartition(Iterable longs, Collector out) throws Exception { - HashSet uniq = new HashSet<>(); - for (Long l : longs) { - uniq.add(l); - } - for (Long l : uniq) { - out.collect(l); - } - } - } - - private static class UniqueNestedPojoLongMapper implements MapPartitionFunction { - private static final long serialVersionUID = 1L; - - @Override - public void mapPartition(Iterable records, Collector out) throws Exception { - HashSet uniq = new HashSet<>(); - for (POJO t : records) { - uniq.add(t.nestedPojo.longNumber); - } - for (Long l : uniq) { - out.collect(l); - } - } - } - - private static class PartitionIndexMapper - extends RichMapFunction> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 map(Long value) throws Exception { - return new Tuple2<>(this.getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), 1); - } - } - - @Test - public void testRangePartitionerOnSequenceData() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSource dataSource = env.generateSequence(0, 10000); - KeySelector keyExtractor = new ObjectSelfKeySelector(); - - MapPartitionFunction> minMaxSelector = - new MinMaxSelector<>(new LongComparator(true)); - - Comparator> tuple2Comparator = - new Tuple2Comparator(new LongComparator(true)); - - List> collected = - dataSource.partitionByRange(keyExtractor).mapPartition(minMaxSelector).collect(); - Collections.sort(collected, tuple2Comparator); - - long previousMax = -1; - for (Tuple2 tuple2 : collected) { - if (previousMax == -1) { - previousMax = tuple2.f1; - } else { - long currentMin = tuple2.f0; - assertTrue(tuple2.f0 < tuple2.f1); - assertEquals(previousMax + 1, currentMin); - previousMax = tuple2.f1; - } - } - } - - @Test(expected = InvalidProgramException.class) - public void testRangePartitionInIteration() throws Exception { - - // does not apply for collection execution - if (super.mode == TestExecutionMode.COLLECTION) { - throw new InvalidProgramException("Does not apply for collection execution"); - } - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSource source = env.generateSequence(0, 10000); - - DataSet> tuples = - source.map( - new MapFunction>() { - @Override - public Tuple2 map(Long v) throws Exception { - return new Tuple2<>(v, Long.toString(v)); - } - }); - - DeltaIteration, Tuple2> it = - tuples.iterateDelta(tuples, 10, 0); - DataSet> body = - it.getWorkset() - .partitionByRange( - 1) // Verify that range partition is not allowed in iteration - .join(it.getSolutionSet()) - .where(0) - .equalTo(0) - .projectFirst(0) - .projectSecond(1); - DataSet> result = it.closeWith(body, body); - - result.collect(); // should fail - } - - @Test - public void testRangePartitionerOnSequenceDataWithOrders() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet> dataSet = - env.generateSequence(0, 10000) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) throws Exception { - return new Tuple2<>(value / 5000, value % 5000); - } - }); - - final Tuple2Comparator tuple2Comparator = - new Tuple2Comparator<>(new LongComparator(true), new LongComparator(false)); - - MinMaxSelector> minMaxSelector = new MinMaxSelector<>(tuple2Comparator); - - final List, Tuple2>> collected = - dataSet.partitionByRange(0, 1) - .withOrders(Order.ASCENDING, Order.DESCENDING) - .mapPartition(minMaxSelector) - .collect(); - - Collections.sort(collected, new Tuple2Comparator<>(tuple2Comparator)); - - Tuple2 previousMax = null; - for (Tuple2, Tuple2> tuple2 : collected) { - assertTrue( - "Min element in each partition should be smaller than max.", - tuple2Comparator.compare(tuple2.f0, tuple2.f1) <= 0); - if (previousMax == null) { - previousMax = tuple2.f1; - } else { - assertTrue( - "Partitions overlap. Previous max should be smaller than current min.", - tuple2Comparator.compare(previousMax, tuple2.f0) < 0); - if (previousMax.f0.equals(tuple2.f0.f0)) { - // check that ordering on the second key is correct - assertEquals( - "Ordering on the second field should be continous.", - previousMax.f1 - 1, - tuple2.f0.f1.longValue()); - } - previousMax = tuple2.f1; - } - } - } - - @Test - public void testRangePartitionerOnSequenceNestedDataWithOrders() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - final DataSet, Long>> dataSet = - env.generateSequence(0, 10000) - .map( - new MapFunction, Long>>() { - @Override - public Tuple2, Long> map(Long value) - throws Exception { - return new Tuple2<>( - new Tuple2<>(value / 5000, value % 5000), value); - } - }); - - final Tuple2Comparator tuple2Comparator = - new Tuple2Comparator<>(new LongComparator(true), new LongComparator(true)); - MinMaxSelector> minMaxSelector = new MinMaxSelector<>(tuple2Comparator); - - final List, Tuple2>> collected = - dataSet.partitionByRange(0) - .withOrders(Order.ASCENDING) - .mapPartition( - new MapPartitionFunction< - Tuple2, Long>, Tuple2>() { - @Override - public void mapPartition( - Iterable, Long>> values, - Collector> out) - throws Exception { - for (Tuple2, Long> value : values) { - out.collect(value.f0); - } - } - }) - .mapPartition(minMaxSelector) - .collect(); - - Collections.sort(collected, new Tuple2Comparator<>(tuple2Comparator)); - - Tuple2 previousMax = null; - for (Tuple2, Tuple2> tuple2 : collected) { - assertTrue( - "Min element in each partition should be smaller than max.", - tuple2Comparator.compare(tuple2.f0, tuple2.f1) <= 0); - if (previousMax == null) { - previousMax = tuple2.f1; - } else { - assertTrue( - "Partitions overlap. Previous max should be smaller than current min.", - tuple2Comparator.compare(previousMax, tuple2.f0) < 0); - if (previousMax.f0.equals(tuple2.f0.f0)) { - assertEquals( - "Ordering on the second field should be continous.", - previousMax.f1 + 1, - tuple2.f0.f1.longValue()); - } - previousMax = tuple2.f1; - } - } - } - - @Test - public void testRangePartitionerWithKeySelectorOnSequenceNestedDataWithOrders() - throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - final DataSet> dataSet = - env.generateSequence(0, 10000) - .map( - new MapFunction>() { - @Override - public Tuple2 map(Long value) - throws Exception { - return new Tuple2<>( - new ComparablePojo(value / 5000, value % 5000), - value); - } - }); - - final List> collected = - dataSet.partitionByRange( - new KeySelector, ComparablePojo>() { - @Override - public ComparablePojo getKey(Tuple2 value) - throws Exception { - return value.f0; - } - }) - .withOrders(Order.ASCENDING) - .mapPartition(new MinMaxSelector<>(new ComparablePojoComparator())) - .mapPartition(new ExtractComparablePojo()) - .collect(); - - final Comparator> pojoComparator = - new Comparator>() { - @Override - public int compare( - Tuple2 o1, - Tuple2 o2) { - return o1.f0.compareTo(o2.f1); - } - }; - Collections.sort(collected, pojoComparator); - - ComparablePojo previousMax = null; - for (Tuple2 element : collected) { - assertTrue( - "Min element in each partition should be smaller than max.", - element.f0.compareTo(element.f1) <= 0); - if (previousMax == null) { - previousMax = element.f1; - } else { - assertTrue( - "Partitions overlap. Previous max should be smaller than current min.", - previousMax.compareTo(element.f0) < 0); - if (previousMax.first.equals(element.f0.first)) { - assertEquals( - "Ordering on the second field should be continous.", - previousMax.second - 1, - element.f0.second.longValue()); - } - previousMax = element.f1; - } - } - } - - private static class ExtractComparablePojo - implements MapPartitionFunction< - Tuple2, Tuple2>, - Tuple2> { - - @Override - public void mapPartition( - Iterable, Tuple2>> values, - Collector> out) - throws Exception { - for (Tuple2, Tuple2> value : - values) { - out.collect(new Tuple2<>(value.f0.f0, value.f1.f0)); - } - } - } - - private static class ComparablePojoComparator - implements Comparator>, Serializable { - - @Override - public int compare(Tuple2 o1, Tuple2 o2) { - return o1.f0.compareTo(o2.f0); - } - } - - /** A comparable POJO. */ - public static class ComparablePojo implements Comparable { - private Long first; - private Long second; - - public Long getFirst() { - return first; - } - - public void setFirst(Long first) { - this.first = first; - } - - public Long getSecond() { - return second; - } - - public void setSecond(Long second) { - this.second = second; - } - - public ComparablePojo(Long first, Long second) { - this.first = first; - this.second = second; - } - - public ComparablePojo() {} - - @Override - public int compareTo(ComparablePojo o) { - final int firstResult = Long.compare(this.first, o.first); - if (firstResult == 0) { - return (-1) * Long.compare(this.second, o.second); - } - - return firstResult; - } - } - - private static class ObjectSelfKeySelector implements KeySelector { - @Override - public Long getKey(Long value) throws Exception { - return value; - } - } - - private static class MinMaxSelector implements MapPartitionFunction> { - - private final Comparator comparator; - - public MinMaxSelector(Comparator comparator) { - this.comparator = comparator; - } - - @Override - public void mapPartition(Iterable values, Collector> out) throws Exception { - Iterator itr = values.iterator(); - if (!itr.hasNext()) { - return; - } - T min = itr.next(); - T max = min; - T value; - while (itr.hasNext()) { - value = itr.next(); - if (comparator.compare(value, min) < 0) { - min = value; - } - if (comparator.compare(value, max) > 0) { - max = value; - } - } - - Tuple2 result = new Tuple2<>(min, max); - out.collect(result); - } - } - - private static class Tuple2Comparator implements Comparator>, Serializable { - - private final Comparator firstComparator; - private final Comparator secondComparator; - - public Tuple2Comparator(Comparator comparator) { - this(comparator, comparator); - } - - public Tuple2Comparator(Comparator firstComparator, Comparator secondComparator) { - this.firstComparator = firstComparator; - this.secondComparator = secondComparator; - } - - @Override - public int compare(Tuple2 first, Tuple2 second) { - long result = firstComparator.compare(first.f0, second.f0); - if (result > 0) { - return 1; - } else if (result < 0) { - return -1; - } - - result = secondComparator.compare(first.f1, second.f1); - if (result > 0) { - return 1; - } else if (result < 0) { - return -1; - } - - return 0; - } - } - - private static class LongComparator implements Comparator, Serializable { - - private final boolean ascending; - - public LongComparator(boolean ascending) { - this.ascending = ascending; - } - - @Override - public int compare(Long o1, Long o2) { - if (ascending) { - return Long.compare(o1, o2); - } else { - return (-1) * Long.compare(o1, o2); - } - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java deleted file mode 100644 index 1caf6ee71aedc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java +++ /dev/null @@ -1,67 +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.operators; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link DataSet#project}. */ -public class ProjectITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - /* - * Projection with tuple fields indexes - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> projDs = ds.project(3, 4, 2); - List> result = projDs.collect(); - - String expectedResult = - "Hallo,1,0\n" - + "Hallo Welt,2,1\n" - + "Hallo Welt wie,1,2\n" - + "Hallo Welt wie gehts?,2,3\n" - + "ABC,2,4\n" - + "BCD,3,5\n" - + "CDE,2,6\n" - + "DEF,1,7\n" - + "EFG,1,8\n" - + "FGH,2,9\n" - + "GHI,1,10\n" - + "HIJ,3,11\n" - + "IJK,3,12\n" - + "JKL,2,13\n" - + "KLM,2,14\n"; - - compareResultAsTuples(result, expectedResult); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java deleted file mode 100644 index 1faed585c4507..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ /dev/null @@ -1,526 +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.operators; - -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.functions.RichReduceFunction; -import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint; -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.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; -import org.apache.flink.test.operators.util.CollectionDataSets.PojoWithDateAndEnum; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; -import java.util.Date; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link ReduceFunction} and {@link RichReduceFunction}. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class ReduceITCase extends MultipleProgramsTestBaseJUnit4 { - - public ReduceITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testReduceOnTuplesWithKeyFieldSelector() throws Exception { - /* - * Reduce on tuples with key field selector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1).reduce(new Tuple3Reduce("B-)")); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "5,2,B-)\n" - + "15,3,B-)\n" - + "34,4,B-)\n" - + "65,5,B-)\n" - + "111,6,B-)\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testReduceOnTupleWithMultipleKeyFieldSelectors() throws Exception { - /* - * Reduce on tuples with multiple key field selectors - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(4, 0) - .reduce( - (in1, in2) -> { - Tuple5 out = - new Tuple5<>(); - out.setFields(in1.f0, in1.f1 + in2.f1, 0, "P-)", in1.f4); - return out; - }); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "2,3,2,Hallo Welt wie,1\n" - + "2,2,1,Hallo Welt,2\n" - + "3,9,0,P-),2\n" - + "3,6,5,BCD,3\n" - + "4,17,0,P-),1\n" - + "4,17,0,P-),2\n" - + "5,11,10,GHI,1\n" - + "5,29,0,P-),2\n" - + "5,25,0,P-),3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testReduceOnTuplesWithKeyExtractor() throws Exception { - /* - * Reduce on tuples with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new KeySelector1()).reduce(new Tuple3Reduce("B-)")); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" - + "5,2,B-)\n" - + "15,3,B-)\n" - + "34,4,B-)\n" - + "65,5,B-)\n" - + "111,6,B-)\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector1 implements KeySelector, Long> { - private static final long serialVersionUID = 1L; - - @Override - public Long getKey(Tuple3 in) { - return in.f1; - } - } - - @Test - public void testReduceOnCustomTypeWithKeyExtractor() throws Exception { - /* - * Reduce on custom type with key extractor - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = - ds.groupBy(new KeySelector2()).reduce(new CustomTypeReduce()); - - List result = reduceDs.collect(); - - String expected = - "1,0,Hi\n" - + "2,3,Hello!\n" - + "3,12,Hello!\n" - + "4,30,Hello!\n" - + "5,60,Hello!\n" - + "6,105,Hello!\n"; - - compareResultAsText(result, expected); - } - - private static class KeySelector2 implements KeySelector { - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(CustomType in) { - return in.myInt; - } - } - - @Test - public void testAllReduceForTuple() throws Exception { - /* - * All-reduce for tuple - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds.reduce(new AllAddingTuple3Reduce()); - - List> result = reduceDs.collect(); - - String expected = "231,91,Hello World\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testAllReduceForCustomTypes() throws Exception { - /* - * All-reduce for custom types - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getCustomTypeDataSet(env); - DataSet reduceDs = ds.reduce(new AllAddingCustomTypeReduce()); - - List result = reduceDs.collect(); - - String expected = "91,210,Hello!"; - - compareResultAsText(result, expected); - } - - @Test - public void testReduceWithBroadcastSet() throws Exception { - /* - * Reduce with broadcast set - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet intDs = CollectionDataSets.getIntegerDataSet(env); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(1).reduce(new BCTuple3Reduce()).withBroadcastSet(intDs, "ints"); - - List> result = reduceDs.collect(); - - String expected = - "1,1,Hi\n" + "5,2,55\n" + "15,3,55\n" + "34,4,55\n" + "65,5,55\n" + "111,6,55\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testReduceATupleReturningKeySelector() throws Exception { - /* - * Reduce with a Tuple-returning KeySelector - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy(new KeySelector3()).reduce(new Tuple5Reduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "2,3,2,Hallo Welt wie,1\n" - + "2,2,1,Hallo Welt,2\n" - + "3,9,0,P-),2\n" - + "3,6,5,BCD,3\n" - + "4,17,0,P-),1\n" - + "4,17,0,P-),2\n" - + "5,11,10,GHI,1\n" - + "5,29,0,P-),2\n" - + "5,25,0,P-),3\n"; - - compareResultAsTuples(result, expected); - } - - private static class KeySelector3 - implements KeySelector< - Tuple5, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 getKey(Tuple5 t) { - return new Tuple2(t.f0, t.f4); - } - } - - @Test - public void testReduceOnTupleWithMultipleKeyExpressions() throws Exception { - /* - * Case 2 with String-based field expression - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy("f4", "f0").reduce(new Tuple5Reduce()); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "2,3,2,Hallo Welt wie,1\n" - + "2,2,1,Hallo Welt,2\n" - + "3,9,0,P-),2\n" - + "3,6,5,BCD,3\n" - + "4,17,0,P-),1\n" - + "4,17,0,P-),2\n" - + "5,11,10,GHI,1\n" - + "5,29,0,P-),2\n" - + "5,25,0,P-),3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testReduceOnTupleWithMultipleKeyExpressionsWithHashHint() throws Exception { - /* - * Case 2 with String-based field expression - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - DataSet> reduceDs = - ds.groupBy("f4", "f0").reduce(new Tuple5Reduce()).setCombineHint(CombineHint.HASH); - - List> result = reduceDs.collect(); - - String expected = - "1,1,0,Hallo,1\n" - + "2,3,2,Hallo Welt wie,1\n" - + "2,2,1,Hallo Welt,2\n" - + "3,9,0,P-),2\n" - + "3,6,5,BCD,3\n" - + "4,17,0,P-),1\n" - + "4,17,0,P-),2\n" - + "5,11,10,GHI,1\n" - + "5,29,0,P-),2\n" - + "5,25,0,P-),3\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testSupportForDataAndEnumSerialization() throws Exception { - /** Test support for Date and enum serialization */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet ds = env.generateSequence(0, 2).map(new Mapper1()); - ds = ds.union(CollectionDataSets.getPojoWithDateAndEnum(env)); - - DataSet res = ds.groupBy("group").reduceGroup(new GroupReducer1()); - - List result = res.collect(); - - String expected = "ok\nok"; - - compareResultAsText(result, expected); - } - - private static class Mapper1 implements MapFunction { - @Override - public PojoWithDateAndEnum map(Long value) throws Exception { - int l = value.intValue(); - switch (l) { - case 0: - PojoWithDateAndEnum one = new PojoWithDateAndEnum(); - one.group = "a"; - one.date = new Date(666); - one.cat = CollectionDataSets.Category.CAT_A; - return one; - case 1: - PojoWithDateAndEnum two = new PojoWithDateAndEnum(); - two.group = "a"; - two.date = new Date(666); - two.cat = CollectionDataSets.Category.CAT_A; - return two; - case 2: - PojoWithDateAndEnum three = new PojoWithDateAndEnum(); - three.group = "b"; - three.date = new Date(666); - three.cat = CollectionDataSets.Category.CAT_B; - return three; - } - throw new RuntimeException("Unexpected value for l=" + l); - } - } - - private static class GroupReducer1 - implements GroupReduceFunction { - private static final long serialVersionUID = 1L; - - @Override - public void reduce(Iterable values, Collector out) - throws Exception { - for (PojoWithDateAndEnum val : values) { - if (val.cat == CollectionDataSets.Category.CAT_A) { - Assert.assertEquals("a", val.group); - } else if (val.cat == CollectionDataSets.Category.CAT_B) { - Assert.assertEquals("b", val.group); - } else { - Assert.fail("error. Cat = " + val.cat); - } - Assert.assertEquals(666, val.date.getTime()); - } - out.collect("ok"); - } - } - - private static class Tuple3Reduce implements ReduceFunction> { - private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); - private final String f2Replace; - - public Tuple3Reduce() { - this.f2Replace = null; - } - - public Tuple3Reduce(String f2Replace) { - this.f2Replace = f2Replace; - } - - @Override - public Tuple3 reduce( - Tuple3 in1, Tuple3 in2) - throws Exception { - - if (f2Replace == null) { - out.setFields(in1.f0 + in2.f0, in1.f1, in1.f2); - } else { - out.setFields(in1.f0 + in2.f0, in1.f1, this.f2Replace); - } - return out; - } - } - - private static class Tuple5Reduce - implements ReduceFunction> { - private static final long serialVersionUID = 1L; - private final Tuple5 out = - new Tuple5(); - - @Override - public Tuple5 reduce( - Tuple5 in1, - Tuple5 in2) - throws Exception { - - out.setFields(in1.f0, in1.f1 + in2.f1, 0, "P-)", in1.f4); - return out; - } - } - - private static class CustomTypeReduce implements ReduceFunction { - private static final long serialVersionUID = 1L; - private final CustomType out = new CustomType(); - - @Override - public CustomType reduce(CustomType in1, CustomType in2) throws Exception { - - out.myInt = in1.myInt; - out.myLong = in1.myLong + in2.myLong; - out.myString = "Hello!"; - return out; - } - } - - private static class AllAddingTuple3Reduce - implements ReduceFunction> { - private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); - - @Override - public Tuple3 reduce( - Tuple3 in1, Tuple3 in2) - throws Exception { - - out.setFields(in1.f0 + in2.f0, in1.f1 + in2.f1, "Hello World"); - return out; - } - } - - private static class AllAddingCustomTypeReduce implements ReduceFunction { - private static final long serialVersionUID = 1L; - private final CustomType out = new CustomType(); - - @Override - public CustomType reduce(CustomType in1, CustomType in2) throws Exception { - - out.myInt = in1.myInt + in2.myInt; - out.myLong = in1.myLong + in2.myLong; - out.myString = "Hello!"; - return out; - } - } - - private static class BCTuple3Reduce extends RichReduceFunction> { - private static final long serialVersionUID = 1L; - private final Tuple3 out = new Tuple3(); - private String f2Replace = ""; - - @Override - public void open(OpenContext openContext) { - - Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); - int sum = 0; - for (Integer i : ints) { - sum += i; - } - f2Replace = sum + ""; - } - - @Override - public Tuple3 reduce( - Tuple3 in1, Tuple3 in2) - throws Exception { - - out.setFields(in1.f0 + in2.f0, in1.f1, this.f2Replace); - return out; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java deleted file mode 100644 index 61b237f2d85b9..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java +++ /dev/null @@ -1,342 +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.operators; - -import org.apache.flink.api.common.functions.CombineFunction; -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.UnsortedGrouping; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link GroupCombineFunction}. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class ReduceWithCombinerITCase extends MultipleProgramsTestBaseJUnit4 { - - public ReduceWithCombinerITCase(TestExecutionMode mode) { - super(TestExecutionMode.CLUSTER); - } - - @Test - public void testReduceOnNonKeyedDataset() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - // creates the input data and distributes them evenly among the available downstream tasks - DataSet> input = createNonKeyedInput(env); - List> actual = - input.reduceGroup(new NonKeyedCombReducer()).collect(); - String expected = "10,true\n"; - - compareResultAsTuples(actual, expected); - } - - @Test - public void testForkingReduceOnNonKeyedDataset() throws Exception { - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - // creates the input data and distributes them evenly among the available downstream tasks - DataSet> input = createNonKeyedInput(env); - - DataSet> r1 = input.reduceGroup(new NonKeyedCombReducer()); - DataSet> r2 = input.reduceGroup(new NonKeyedGroupCombReducer()); - - List> actual = r1.union(r2).collect(); - String expected = "10,true\n10,true\n"; - compareResultAsTuples(actual, expected); - } - - @Test - public void testReduceOnKeyedDataset() throws Exception { - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - // creates the input data and distributes them evenly among the available downstream tasks - DataSet> input = createKeyedInput(env); - List> actual = - input.groupBy(0).reduceGroup(new KeyedCombReducer()).collect(); - String expected = "k1,6,true\nk2,4,true\n"; - - compareResultAsTuples(actual, expected); - } - - @Test - public void testReduceOnKeyedDatasetWithSelector() throws Exception { - - // set up the execution environment - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - // creates the input data and distributes them evenly among the available downstream tasks - DataSet> input = createKeyedInput(env); - - List> actual = - input.groupBy(new KeySelectorX()).reduceGroup(new KeyedCombReducer()).collect(); - String expected = "k1,6,true\nk2,4,true\n"; - - compareResultAsTuples(actual, expected); - } - - @Test - public void testForkingReduceOnKeyedDataset() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - // creates the input data and distributes them evenly among the available downstream tasks - DataSet> input = createKeyedInput(env); - - UnsortedGrouping> counts = input.groupBy(0); - - DataSet> r1 = counts.reduceGroup(new KeyedCombReducer()); - DataSet> r2 = - counts.reduceGroup(new KeyedGroupCombReducer()); - - List> actual = r1.union(r2).collect(); - String expected = "k1,6,true\n" + "k2,4,true\n" + "k1,6,true\n" + "k2,4,true\n"; - compareResultAsTuples(actual, expected); - } - - @Test - public void testForkingReduceOnKeyedDatasetWithSelection() throws Exception { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - // creates the input data and distributes them evenly among the available downstream tasks - DataSet> input = createKeyedInput(env); - - UnsortedGrouping> counts = - input.groupBy(new KeySelectorX()); - - DataSet> r1 = counts.reduceGroup(new KeyedCombReducer()); - DataSet> r2 = - counts.reduceGroup(new KeyedGroupCombReducer()); - - List> actual = r1.union(r2).collect(); - String expected = "k1,6,true\n" + "k2,4,true\n" + "k1,6,true\n" + "k2,4,true\n"; - - compareResultAsTuples(actual, expected); - } - - private DataSet> createNonKeyedInput(ExecutionEnvironment env) { - return env.fromCollection( - Arrays.asList( - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false), - new Tuple2<>(1, false))) - .rebalance(); - } - - private static class NonKeyedCombReducer - implements CombineFunction, Tuple2>, - GroupReduceFunction, Tuple2> { - - @Override - public Tuple2 combine(Iterable> values) - throws Exception { - int sum = 0; - boolean flag = true; - - for (Tuple2 tuple : values) { - sum += tuple.f0; - flag &= !tuple.f1; - } - return new Tuple2<>(sum, flag); - } - - @Override - public void reduce( - Iterable> values, Collector> out) - throws Exception { - int sum = 0; - boolean flag = true; - for (Tuple2 tuple : values) { - sum += tuple.f0; - flag &= tuple.f1; - } - out.collect(new Tuple2<>(sum, flag)); - } - } - - private static class NonKeyedGroupCombReducer - implements GroupCombineFunction, Tuple2>, - GroupReduceFunction, Tuple2> { - - @Override - public void reduce( - Iterable> values, Collector> out) - throws Exception { - int sum = 0; - boolean flag = true; - for (Tuple2 tuple : values) { - sum += tuple.f0; - flag &= tuple.f1; - } - out.collect(new Tuple2<>(sum, flag)); - } - - @Override - public void combine( - Iterable> values, Collector> out) - throws Exception { - int sum = 0; - boolean flag = true; - for (Tuple2 tuple : values) { - sum += tuple.f0; - flag &= !tuple.f1; - } - out.collect(new Tuple2<>(sum, flag)); - } - } - - private DataSet> createKeyedInput(ExecutionEnvironment env) { - return env.fromCollection( - Arrays.asList( - new Tuple3<>("k1", 1, false), - new Tuple3<>("k1", 1, false), - new Tuple3<>("k1", 1, false), - new Tuple3<>("k2", 1, false), - new Tuple3<>("k1", 1, false), - new Tuple3<>("k1", 1, false), - new Tuple3<>("k2", 1, false), - new Tuple3<>("k2", 1, false), - new Tuple3<>("k1", 1, false), - new Tuple3<>("k2", 1, false))) - .rebalance(); - } - - private static class KeySelectorX - implements KeySelector, String> { - private static final long serialVersionUID = 1L; - - @Override - public String getKey(Tuple3 in) { - return in.f0; - } - } - - private class KeyedCombReducer - implements CombineFunction< - Tuple3, Tuple3>, - GroupReduceFunction< - Tuple3, Tuple3> { - - @Override - public Tuple3 combine( - Iterable> values) throws Exception { - String key = null; - int sum = 0; - boolean flag = true; - - for (Tuple3 tuple : values) { - key = (key == null) ? tuple.f0 : key; - sum += tuple.f1; - flag &= !tuple.f2; - } - return new Tuple3<>(key, sum, flag); - } - - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - String key = null; - int sum = 0; - boolean flag = true; - - for (Tuple3 tuple : values) { - key = (key == null) ? tuple.f0 : key; - sum += tuple.f1; - flag &= tuple.f2; - } - out.collect(new Tuple3<>(key, sum, flag)); - } - } - - private class KeyedGroupCombReducer - implements GroupCombineFunction< - Tuple3, Tuple3>, - GroupReduceFunction< - Tuple3, Tuple3> { - - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception { - String key = null; - int sum = 0; - boolean flag = true; - - for (Tuple3 tuple : values) { - key = (key == null) ? tuple.f0 : key; - sum += tuple.f1; - flag &= !tuple.f2; - } - out.collect(new Tuple3<>(key, sum, flag)); - } - - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - String key = null; - int sum = 0; - boolean flag = true; - - for (Tuple3 tuple : values) { - key = (key == null) ? tuple.f0 : key; - sum += tuple.f1; - flag &= tuple.f2; - } - out.collect(new Tuple3<>(key, sum, flag)); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java deleted file mode 100644 index 9f00f7fd980e0..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java +++ /dev/null @@ -1,117 +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.operators; - -import org.apache.flink.api.common.functions.RichMapPartitionFunction; -import org.apache.flink.api.common.io.GenericInputFormat; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.RpcOptions; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.runtime.testutils.MiniClusterResource; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.util.Collector; -import org.apache.flink.util.TestLogger; - -import org.junit.ClassRule; -import org.junit.Test; - -import java.io.IOException; -import java.net.URI; -import java.time.Duration; -import java.util.List; - -import static org.junit.Assert.assertEquals; - -/** Integration tests for {@link org.apache.flink.api.java.RemoteEnvironment}. */ -@SuppressWarnings("serial") -public class RemoteEnvironmentITCase extends TestLogger { - - private static final int TM_SLOTS = 4; - - private static final int USER_DOP = 2; - - private static final Duration VALID_STARTUP_TIMEOUT = Duration.ofSeconds(100); - - @ClassRule - public static final MiniClusterResource MINI_CLUSTER_RESOURCE = - new MiniClusterResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberSlotsPerTaskManager(TM_SLOTS) - .build()); - - /** Ensure that the program parallelism can be set even if the configuration is supplied. */ - @Test - public void testUserSpecificParallelism() throws Exception { - Configuration config = new Configuration(); - config.set(RpcOptions.STARTUP_TIMEOUT, VALID_STARTUP_TIMEOUT); - - final URI restAddress = MINI_CLUSTER_RESOURCE.getRestAddress(); - final String hostname = restAddress.getHost(); - final int port = restAddress.getPort(); - - final ExecutionEnvironment env = - ExecutionEnvironment.createRemoteEnvironment(hostname, port, config); - env.setParallelism(USER_DOP); - - DataSet result = - env.createInput(new ParallelismDependentInputFormat()) - .rebalance() - .mapPartition( - new RichMapPartitionFunction() { - @Override - public void mapPartition( - Iterable values, Collector out) - throws Exception { - out.collect( - getRuntimeContext() - .getTaskInfo() - .getIndexOfThisSubtask()); - } - }); - List resultCollection = result.collect(); - assertEquals(USER_DOP, resultCollection.size()); - } - - private static class ParallelismDependentInputFormat extends GenericInputFormat { - - private transient boolean emitted; - - @Override - public GenericInputSplit[] createInputSplits(int numSplits) throws IOException { - assertEquals(USER_DOP, numSplits); - return super.createInputSplits(numSplits); - } - - @Override - public boolean reachedEnd() { - return emitted; - } - - @Override - public Integer nextRecord(Integer reuse) { - if (emitted) { - return null; - } - emitted = true; - return 1; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java deleted file mode 100644 index 7eb892a958611..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java +++ /dev/null @@ -1,130 +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.operators; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.io.ReplicatingInputFormat; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.ParallelIteratorInputFormat; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.NumberSequenceIterator; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; - -/** Tests for replicating DataSources. */ -@RunWith(Parameterized.class) -public class ReplicatingDataSourceITCase extends MultipleProgramsTestBaseJUnit4 { - - public ReplicatingDataSourceITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testReplicatedSourceToJoin() throws Exception { - /* - * Test replicated source going into join - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> source1 = - env.createInput( - new ReplicatingInputFormat( - new ParallelIteratorInputFormat( - new NumberSequenceIterator(0L, 1000L))), - BasicTypeInfo.LONG_TYPE_INFO) - .map(new ToTuple()); - DataSet> source2 = env.generateSequence(0L, 1000L).map(new ToTuple()); - - DataSet pairs = source1.join(source2).where(0).equalTo(0).projectFirst(0).sum(0); - - List result = pairs.collect(); - - String expectedResult = "(500500)"; - - compareResultAsText(result, expectedResult); - } - - @Test - public void testReplicatedSourceToCross() throws Exception { - /* - * Test replicated source going into cross - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> source1 = - env.createInput( - new ReplicatingInputFormat( - new ParallelIteratorInputFormat( - new NumberSequenceIterator(0L, 1000L))), - BasicTypeInfo.LONG_TYPE_INFO) - .map(new ToTuple()); - DataSet> source2 = env.generateSequence(0L, 1000L).map(new ToTuple()); - - DataSet> pairs = - source1.cross(source2) - .filter( - new FilterFunction, Tuple1>>() { - @Override - public boolean filter(Tuple2, Tuple1> value) - throws Exception { - return value.f0.f0.equals(value.f1.f0); - } - }) - .map( - new MapFunction< - Tuple2, Tuple1>, Tuple1>() { - @Override - public Tuple1 map( - Tuple2, Tuple1> value) - throws Exception { - return value.f0; - } - }) - .sum(0); - - List> result = pairs.collect(); - - String expectedResult = "(500500)"; - - compareResultAsText(result, expectedResult); - } - - private static class ToTuple implements MapFunction> { - - @Override - public Tuple1 map(Long value) throws Exception { - return new Tuple1(value); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java deleted file mode 100644 index 23d62be9f1079..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java +++ /dev/null @@ -1,182 +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.operators; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.operators.FlatMapOperator; -import org.apache.flink.api.java.operators.MapPartitionOperator; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; -import java.util.Random; - -import static org.apache.flink.test.util.TestBaseUtils.containsResultAsText; -import static org.junit.Assert.assertEquals; - -/** Integration tests for {@link DataSetUtils#sample}. */ -@SuppressWarnings("serial") -@RunWith(Parameterized.class) -public class SampleITCase extends MultipleProgramsTestBaseJUnit4 { - - private static final Random RNG = new Random(); - - public SampleITCase(TestExecutionMode mode) { - super(mode); - } - - @Before - public void initiate() { - ExecutionEnvironment.getExecutionEnvironment().setParallelism(5); - } - - @Test - public void testSamplerWithFractionWithoutReplacement() throws Exception { - verifySamplerWithFractionWithoutReplacement(0d); - verifySamplerWithFractionWithoutReplacement(0.2d); - verifySamplerWithFractionWithoutReplacement(1.0d); - } - - @Test - public void testSamplerWithFractionWithReplacement() throws Exception { - verifySamplerWithFractionWithReplacement(0d); - verifySamplerWithFractionWithReplacement(0.2d); - verifySamplerWithFractionWithReplacement(1.0d); - verifySamplerWithFractionWithReplacement(2.0d); - } - - @Test - public void testSamplerWithSizeWithoutReplacement() throws Exception { - verifySamplerWithFixedSizeWithoutReplacement(0); - verifySamplerWithFixedSizeWithoutReplacement(2); - verifySamplerWithFixedSizeWithoutReplacement(21); - } - - @Test - public void testSamplerWithSizeWithReplacement() throws Exception { - verifySamplerWithFixedSizeWithReplacement(0); - verifySamplerWithFixedSizeWithReplacement(2); - verifySamplerWithFixedSizeWithReplacement(21); - } - - private void verifySamplerWithFractionWithoutReplacement(double fraction) throws Exception { - verifySamplerWithFractionWithoutReplacement(fraction, RNG.nextLong()); - } - - private void verifySamplerWithFractionWithoutReplacement(double fraction, long seed) - throws Exception { - verifySamplerWithFraction(false, fraction, seed); - } - - private void verifySamplerWithFractionWithReplacement(double fraction) throws Exception { - verifySamplerWithFractionWithReplacement(fraction, RNG.nextLong()); - } - - private void verifySamplerWithFractionWithReplacement(double fraction, long seed) - throws Exception { - verifySamplerWithFraction(true, fraction, seed); - } - - private void verifySamplerWithFraction(boolean withReplacement, double fraction, long seed) - throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - FlatMapOperator, String> ds = getSourceDataSet(env); - MapPartitionOperator sampled = - DataSetUtils.sample(ds, withReplacement, fraction, seed); - List result = sampled.collect(); - containsResultAsText(result, getSourceStrings()); - } - - private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples) throws Exception { - verifySamplerWithFixedSizeWithoutReplacement(numSamples, RNG.nextLong()); - } - - private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples, long seed) - throws Exception { - verifySamplerWithFixedSize(false, numSamples, seed); - } - - private void verifySamplerWithFixedSizeWithReplacement(int numSamples) throws Exception { - verifySamplerWithFixedSizeWithReplacement(numSamples, RNG.nextLong()); - } - - private void verifySamplerWithFixedSizeWithReplacement(int numSamples, long seed) - throws Exception { - verifySamplerWithFixedSize(true, numSamples, seed); - } - - private void verifySamplerWithFixedSize(boolean withReplacement, int numSamples, long seed) - throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - FlatMapOperator, String> ds = getSourceDataSet(env); - DataSet sampled = - DataSetUtils.sampleWithSize(ds, withReplacement, numSamples, seed); - List result = sampled.collect(); - assertEquals(numSamples, result.size()); - containsResultAsText(result, getSourceStrings()); - } - - private FlatMapOperator, String> getSourceDataSet( - ExecutionEnvironment env) { - return CollectionDataSets.get3TupleDataSet(env) - .flatMap( - new FlatMapFunction, String>() { - @Override - public void flatMap( - Tuple3 value, Collector out) - throws Exception { - out.collect(value.f2); - } - }); - } - - private String getSourceStrings() { - return "Hi\n" - + "Hello\n" - + "Hello world\n" - + "Hello world, how are you?\n" - + "I am fine.\n" - + "Luke Skywalker\n" - + "Comment#1\n" - + "Comment#2\n" - + "Comment#3\n" - + "Comment#4\n" - + "Comment#5\n" - + "Comment#6\n" - + "Comment#7\n" - + "Comment#8\n" - + "Comment#9\n" - + "Comment#10\n" - + "Comment#11\n" - + "Comment#12\n" - + "Comment#13\n" - + "Comment#14\n" - + "Comment#15\n"; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java deleted file mode 100644 index 062a70fcf3e0d..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java +++ /dev/null @@ -1,384 +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.operators; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.MapPartitionFunction; -import org.apache.flink.api.common.operators.Order; -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.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.operators.util.CollectionDataSets.POJO; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; -import org.apache.flink.util.Collector; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.Serializable; -import java.util.Iterator; -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsText; - -/** Tests for {@link DataSet#sortPartition}. */ -@RunWith(Parameterized.class) -public class SortPartitionITCase extends MultipleProgramsTestBaseJUnit4 { - - public SortPartitionITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testSortPartitionByKeyField() throws Exception { - /* - * Test sort partition on key field - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - List> result = - ds.map(new IdMapper>()) - .setParallelism(4) // parallelize input - .sortPartition(1, Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new Tuple3Checker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionByTwoKeyFields() throws Exception { - /* - * Test sort partition on two key fields - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - List> result = - ds.map(new IdMapper>()) - .setParallelism(2) // parallelize input - .sortPartition(4, Order.ASCENDING) - .sortPartition(2, Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new Tuple5Checker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @SuppressWarnings({"rawtypes", "unchecked"}) - @Test - public void testSortPartitionByFieldExpression() throws Exception { - /* - * Test sort partition on field expression - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - List> result = - ds.map(new IdMapper()) - .setParallelism(4) // parallelize input - .sortPartition("f1", Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new Tuple3Checker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionByTwoFieldExpressions() throws Exception { - /* - * Test sort partition on two field expressions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - DataSet> ds = - CollectionDataSets.get5TupleDataSet(env); - List> result = - ds.map(new IdMapper>()) - .setParallelism(2) // parallelize input - .sortPartition("f4", Order.ASCENDING) - .sortPartition("f2", Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new Tuple5Checker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionByNestedFieldExpression() throws Exception { - /* - * Test sort partition on nested field expressions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet, String>> ds = - CollectionDataSets.getGroupSortedNestedTupleDataSet(env); - List> result = - ds.map(new IdMapper, String>>()) - .setParallelism(3) // parallelize input - .sortPartition("f0.f1", Order.ASCENDING) - .sortPartition("f1", Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new NestedTupleChecker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionPojoByNestedFieldExpression() throws Exception { - /* - * Test sort partition on field expression - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet ds = CollectionDataSets.getMixedPojoDataSet(env); - List> result = - ds.map(new IdMapper()) - .setParallelism(1) // parallelize input - .sortPartition("nestedTupleWithCustom.f1.myString", Order.ASCENDING) - .sortPartition("number", Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new PojoChecker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionParallelismChange() throws Exception { - /* - * Test sort partition with parallelism change - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(3); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - List> result = - ds.sortPartition(1, Order.DESCENDING) - .setParallelism(3) // change parallelism - .mapPartition(new OrderCheckMapper<>(new Tuple3Checker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionWithKeySelector1() throws Exception { - /* - * Test sort partition on an extracted key - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - List> result = - ds.map(new IdMapper>()) - .setParallelism(4) // parallelize input - .sortPartition( - new KeySelector, Long>() { - @Override - public Long getKey(Tuple3 value) - throws Exception { - return value.f1; - } - }, - Order.ASCENDING) - .mapPartition(new OrderCheckMapper<>(new Tuple3AscendingChecker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - @Test - public void testSortPartitionWithKeySelector2() throws Exception { - /* - * Test sort partition on an extracted key - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(4); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - List> result = - ds.map(new IdMapper>()) - .setParallelism(4) // parallelize input - .sortPartition( - new KeySelector< - Tuple3, Tuple2>() { - @Override - public Tuple2 getKey( - Tuple3 value) throws Exception { - return new Tuple2<>(value.f0, value.f1); - } - }, - Order.DESCENDING) - .mapPartition(new OrderCheckMapper<>(new Tuple3Checker())) - .distinct() - .collect(); - - String expected = "(true)\n"; - - compareResultAsText(result, expected); - } - - private interface OrderChecker extends Serializable { - boolean inOrder(T t1, T t2); - } - - @SuppressWarnings("serial") - private static class Tuple3Checker implements OrderChecker> { - @Override - public boolean inOrder(Tuple3 t1, Tuple3 t2) { - return t1.f1 >= t2.f1; - } - } - - @SuppressWarnings("serial") - private static class Tuple3AscendingChecker - implements OrderChecker> { - @Override - public boolean inOrder(Tuple3 t1, Tuple3 t2) { - return t1.f1 <= t2.f1; - } - } - - @SuppressWarnings("serial") - private static class Tuple5Checker - implements OrderChecker> { - @Override - public boolean inOrder( - Tuple5 t1, - Tuple5 t2) { - return t1.f4 < t2.f4 || t1.f4.equals(t2.f4) && t1.f2 >= t2.f2; - } - } - - @SuppressWarnings("serial") - private static class NestedTupleChecker - implements OrderChecker, String>> { - @Override - public boolean inOrder( - Tuple2, String> t1, - Tuple2, String> t2) { - return t1.f0.f1 < t2.f0.f1 || t1.f0.f1.equals(t2.f0.f1) && t1.f1.compareTo(t2.f1) >= 0; - } - } - - @SuppressWarnings("serial") - private static class PojoChecker implements OrderChecker { - @Override - public boolean inOrder(POJO t1, POJO t2) { - return t1.nestedTupleWithCustom.f1.myString.compareTo( - t2.nestedTupleWithCustom.f1.myString) - < 0 - || t1.nestedTupleWithCustom.f1.myString.compareTo( - t2.nestedTupleWithCustom.f1.myString) - == 0 - && t1.number >= t2.number; - } - } - - @SuppressWarnings("unused, serial") - private static class OrderCheckMapper implements MapPartitionFunction> { - - OrderChecker checker; - - public OrderCheckMapper() {} - - public OrderCheckMapper(OrderChecker checker) { - this.checker = checker; - } - - @Override - public void mapPartition(Iterable values, Collector> out) - throws Exception { - - Iterator it = values.iterator(); - if (!it.hasNext()) { - out.collect(new Tuple1<>(true)); - } else { - T last = it.next(); - - while (it.hasNext()) { - T next = it.next(); - if (!checker.inOrder(last, next)) { - out.collect(new Tuple1<>(false)); - return; - } - last = next; - } - out.collect(new Tuple1<>(true)); - } - } - } - - @SuppressWarnings("serial") - private static class IdMapper implements MapFunction { - - @Override - public T map(T value) throws Exception { - return value; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java deleted file mode 100644 index ea6a7c1017953..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java +++ /dev/null @@ -1,97 +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.operators; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** - * Integration tests for {@link org.apache.flink.api.scala.GroupedDataSet#min} and {@link - * org.apache.flink.api.scala.GroupedDataSet#max}. - */ -@RunWith(Parameterized.class) -public class SumMinMaxITCase extends MultipleProgramsTestBaseJUnit4 { - - public SumMinMaxITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testSumMaxAndProject() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> sumDs = ds.sum(0).andMax(1).project(0, 1); - - List> result = sumDs.collect(); - - String expected = "231,6\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testGroupedAggregate() throws Exception { - /* - * Grouped Aggregate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = ds.groupBy(1).sum(0).project(1, 0); - - List> result = aggregateDs.collect(); - - String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; - - compareResultAsTuples(result, expected); - } - - @Test - public void testNestedAggregate() throws Exception { - /* - * Nested Aggregate - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> aggregateDs = ds.groupBy(1).min(0).min(0).project(0); - - List> result = aggregateDs.collect(); - - String expected = "1\n"; - - compareResultAsTuples(result, expected); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java index fc917da7bb0ba..6b3b8f1894f52 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java @@ -25,15 +25,17 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.test.operators.util.CollectionDataSets; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.test.operators.util.CollectionDataStreams; import org.apache.flink.test.util.AbstractTestBaseJUnit4; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Collector; import org.junit.Test; @@ -47,13 +49,15 @@ public class TypeHintITCase extends AbstractTestBaseJUnit4 { @Test public void testIdentityMapWithMissingTypesAndStringTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> identityMapDs = + DataStreamSource> ds = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStream> identityMapDs = ds.map(new Mapper, Tuple3>()) .returns(new TypeHint>() {}); - List> result = identityMapDs.collect(); + List> result = + CollectionUtil.iteratorToList(identityMapDs.executeAndCollect()); String expectedResult = "(2,2,Hello)\n" + "(3,2,Hello world)\n" + "(1,1,Hi)\n"; @@ -62,10 +66,11 @@ public void testIdentityMapWithMissingTypesAndStringTypeHint() throws Exception @Test public void testIdentityMapWithMissingTypesAndTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> identityMapDs = + DataStreamSource> ds = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStream> identityMapDs = ds // all following generics get erased during compilation .map( @@ -77,7 +82,8 @@ public void testIdentityMapWithMissingTypesAndTypeInformationTypeHint() throws E BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO)); - List> result = identityMapDs.collect(); + List> result = + CollectionUtil.iteratorToList(identityMapDs.executeAndCollect()); String expectedResult = "(2,2,Hello)\n" + "(3,2,Hello world)\n" + "(1,1,Hi)\n"; @@ -86,13 +92,14 @@ public void testIdentityMapWithMissingTypesAndTypeInformationTypeHint() throws E @Test public void testFlatMapWithClassTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet identityMapDs = + DataStreamSource> ds = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStream identityMapDs = ds.flatMap(new FlatMapper, Integer>()) .returns(Integer.class); - List result = identityMapDs.collect(); + List result = CollectionUtil.iteratorToList(identityMapDs.executeAndCollect()); String expectedResult = "2\n" + "3\n" + "1\n"; @@ -101,21 +108,23 @@ public void testFlatMapWithClassTypeHint() throws Exception { @Test public void testJoinWithTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet resultDs = + DataStreamSource> ds1 = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStreamSource> ds2 = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStream resultDs = ds1.join(ds2) - .where(0) - .equalTo(0) - .with( + .where(x -> x.f0) + .equalTo(x -> x.f0) + .window(GlobalWindows.createWithEndOfStreamTrigger()) + .apply( new Joiner< Tuple3, Tuple3, - Integer>()) - .returns(BasicTypeInfo.INT_TYPE_INFO); - List result = resultDs.collect(); + Integer>() {}); + List result = CollectionUtil.iteratorToList(resultDs.executeAndCollect()); String expectedResult = "2\n" + "3\n" + "1\n"; @@ -124,70 +133,23 @@ public void testJoinWithTypeInformationTypeHint() throws Exception { @Test public void testFlatJoinWithTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet resultDs = + DataStreamSource> ds1 = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStreamSource> ds2 = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStream resultDs = ds1.join(ds2) - .where(0) - .equalTo(0) - .with( + .where(x -> x.f0) + .equalTo(x -> x.f0) + .window(GlobalWindows.createWithEndOfStreamTrigger()) + .apply( new FlatJoiner< Tuple3, Tuple3, - Integer>()) - .returns(BasicTypeInfo.INT_TYPE_INFO); - List result = resultDs.collect(); - - String expectedResult = "2\n" + "3\n" + "1\n"; - - compareResultAsText(result, expectedResult); - } - - @Test - public void testUnsortedGroupReduceWithTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet resultDs = - ds.groupBy(0) - .reduceGroup(new GroupReducer, Integer>()) - .returns(BasicTypeInfo.INT_TYPE_INFO); - List result = resultDs.collect(); - - String expectedResult = "2\n" + "3\n" + "1\n"; - - compareResultAsText(result, expectedResult); - } - - @Test - public void testSortedGroupReduceWithTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet resultDs = - ds.groupBy(0) - .sortGroup(0, Order.ASCENDING) - .reduceGroup(new GroupReducer, Integer>()) - .returns(BasicTypeInfo.INT_TYPE_INFO); - List result = resultDs.collect(); - - String expectedResult = "2\n" + "3\n" + "1\n"; - - compareResultAsText(result, expectedResult); - } - - @Test - public void testCombineGroupWithTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet resultDs = - ds.groupBy(0) - .combineGroup(new GroupCombiner, Integer>()) - .returns(BasicTypeInfo.INT_TYPE_INFO); - List result = resultDs.collect(); + Integer>() {}); + List result = CollectionUtil.iteratorToList(resultDs.executeAndCollect()); String expectedResult = "2\n" + "3\n" + "1\n"; @@ -196,21 +158,23 @@ public void testCombineGroupWithTypeInformationTypeHint() throws Exception { @Test public void testCoGroupWithTypeInformationTypeHint() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet resultDs = + DataStreamSource> ds1 = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStreamSource> ds2 = + CollectionDataStreams.getSmall3TupleDataSet(env); + DataStream resultDs = ds1.coGroup(ds2) - .where(0) - .equalTo(0) - .with( + .where(x -> x.f0) + .equalTo(x -> x.f0) + .window(GlobalWindows.createWithEndOfStreamTrigger()) + .apply( new CoGrouper< Tuple3, Tuple3, - Integer>()) - .returns(BasicTypeInfo.INT_TYPE_INFO); - List result = resultDs.collect(); + Integer>() {}); + List result = CollectionUtil.iteratorToList(resultDs.executeAndCollect()); String expectedResult = "2\n" + "3\n" + "1\n"; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java deleted file mode 100644 index 4a9c5eb5ce34a..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java +++ /dev/null @@ -1,141 +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.operators; - -import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBaseJUnit4; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultAsTuples; - -/** Integration tests for {@link DataSet#union}. */ -@RunWith(Parameterized.class) -public class UnionITCase extends MultipleProgramsTestBaseJUnit4 { - - private static final String FULL_TUPLE_3_STRING = - "1,1,Hi\n" - + "2,2,Hello\n" - + "3,2,Hello world\n" - + "4,3,Hello world, how are you?\n" - + "5,3,I am fine.\n" - + "6,3,Luke Skywalker\n" - + "7,4,Comment#1\n" - + "8,4,Comment#2\n" - + "9,4,Comment#3\n" - + "10,4,Comment#4\n" - + "11,5,Comment#5\n" - + "12,5,Comment#6\n" - + "13,5,Comment#7\n" - + "14,5,Comment#8\n" - + "15,5,Comment#9\n" - + "16,6,Comment#10\n" - + "17,6,Comment#11\n" - + "18,6,Comment#12\n" - + "19,6,Comment#13\n" - + "20,6,Comment#14\n" - + "21,6,Comment#15\n"; - - public UnionITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testUnion2IdenticalDataSets() throws Exception { - /* - * Union of 2 Same Data Sets - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> unionDs = - ds.union(CollectionDataSets.get3TupleDataSet(env)); - - List> result = unionDs.collect(); - - String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING; - - compareResultAsTuples(result, expected); - } - - @Test - public void testUnion5IdenticalDataSets() throws Exception { - /* - * Union of 5 same Data Sets, with multiple unions - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> unionDs = - ds.union(CollectionDataSets.get3TupleDataSet(env)) - .union(CollectionDataSets.get3TupleDataSet(env)) - .union(CollectionDataSets.get3TupleDataSet(env)) - .union(CollectionDataSets.get3TupleDataSet(env)); - - List> result = unionDs.collect(); - - String expected = - FULL_TUPLE_3_STRING - + FULL_TUPLE_3_STRING - + FULL_TUPLE_3_STRING - + FULL_TUPLE_3_STRING - + FULL_TUPLE_3_STRING; - - compareResultAsTuples(result, expected); - } - - @Test - public void testUnionWithEmptyDataSet() throws Exception { - /* - * Test on union with empty dataset - */ - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // Don't know how to make an empty result in an other way than filtering it - DataSet> empty = - CollectionDataSets.get3TupleDataSet(env).filter(new RichFilter1()); - - DataSet> unionDs = - CollectionDataSets.get3TupleDataSet(env).union(empty); - - List> result = unionDs.collect(); - - String expected = FULL_TUPLE_3_STRING; - - compareResultAsTuples(result, expected); - } - - private static class RichFilter1 extends RichFilterFunction> { - private static final long serialVersionUID = 1L; - - @Override - public boolean filter(Tuple3 value) throws Exception { - return false; - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java deleted file mode 100644 index 9b3ca87ddecdc..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java +++ /dev/null @@ -1,792 +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.operators.util; - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -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.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -import org.apache.hadoop.io.IntWritable; - -import java.io.File; -import java.io.Serializable; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.GregorianCalendar; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import scala.math.BigInt; - -/** - * ####################################################################################################### - * BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA. IF YOU MODIFY THE DATA MAKE SURE YOU CHECK - * THAT ALL TESTS ARE STILL WORKING! - * ####################################################################################################### - */ -public class CollectionDataSets { - - public static DataSet> get3TupleDataSet( - ExecutionEnvironment env) { - - List> data = new ArrayList<>(); - data.add(new Tuple3<>(1, 1L, "Hi")); - data.add(new Tuple3<>(2, 2L, "Hello")); - data.add(new Tuple3<>(3, 2L, "Hello world")); - data.add(new Tuple3<>(4, 3L, "Hello world, how are you?")); - data.add(new Tuple3<>(5, 3L, "I am fine.")); - data.add(new Tuple3<>(6, 3L, "Luke Skywalker")); - data.add(new Tuple3<>(7, 4L, "Comment#1")); - data.add(new Tuple3<>(8, 4L, "Comment#2")); - data.add(new Tuple3<>(9, 4L, "Comment#3")); - data.add(new Tuple3<>(10, 4L, "Comment#4")); - data.add(new Tuple3<>(11, 5L, "Comment#5")); - data.add(new Tuple3<>(12, 5L, "Comment#6")); - data.add(new Tuple3<>(13, 5L, "Comment#7")); - data.add(new Tuple3<>(14, 5L, "Comment#8")); - data.add(new Tuple3<>(15, 5L, "Comment#9")); - data.add(new Tuple3<>(16, 6L, "Comment#10")); - data.add(new Tuple3<>(17, 6L, "Comment#11")); - data.add(new Tuple3<>(18, 6L, "Comment#12")); - data.add(new Tuple3<>(19, 6L, "Comment#13")); - data.add(new Tuple3<>(20, 6L, "Comment#14")); - data.add(new Tuple3<>(21, 6L, "Comment#15")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet> getSmall3TupleDataSet( - ExecutionEnvironment env) { - - List> data = new ArrayList<>(); - data.add(new Tuple3<>(1, 1L, "Hi")); - data.add(new Tuple3<>(2, 2L, "Hello")); - data.add(new Tuple3<>(3, 2L, "Hello world")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet> get5TupleDataSet( - ExecutionEnvironment env) { - - List> data = new ArrayList<>(); - data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L)); - data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L)); - data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L)); - data.add(new Tuple5<>(3, 4L, 3, "Hallo Welt wie gehts?", 2L)); - data.add(new Tuple5<>(3, 5L, 4, "ABC", 2L)); - data.add(new Tuple5<>(3, 6L, 5, "BCD", 3L)); - data.add(new Tuple5<>(4, 7L, 6, "CDE", 2L)); - data.add(new Tuple5<>(4, 8L, 7, "DEF", 1L)); - data.add(new Tuple5<>(4, 9L, 8, "EFG", 1L)); - data.add(new Tuple5<>(4, 10L, 9, "FGH", 2L)); - data.add(new Tuple5<>(5, 11L, 10, "GHI", 1L)); - data.add(new Tuple5<>(5, 12L, 11, "HIJ", 3L)); - data.add(new Tuple5<>(5, 13L, 12, "IJK", 3L)); - data.add(new Tuple5<>(5, 14L, 13, "JKL", 2L)); - data.add(new Tuple5<>(5, 15L, 14, "KLM", 2L)); - - Collections.shuffle(data); - - TupleTypeInfo> type = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet> getSmall5TupleDataSet( - ExecutionEnvironment env) { - - List> data = new ArrayList<>(); - data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L)); - data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L)); - data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L)); - - Collections.shuffle(data); - - TupleTypeInfo> type = - new TupleTypeInfo<>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet, String>> getSmallNestedTupleDataSet( - ExecutionEnvironment env) { - - List, String>> data = new ArrayList<>(); - data.add(new Tuple2<>(new Tuple2<>(1, 1), "one")); - data.add(new Tuple2<>(new Tuple2<>(2, 2), "two")); - data.add(new Tuple2<>(new Tuple2<>(3, 3), "three")); - - TupleTypeInfo, String>> type = - new TupleTypeInfo<>( - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), - BasicTypeInfo.STRING_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet, String>> - getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) { - - List, String>> data = new ArrayList<>(); - data.add(new Tuple2<>(new Tuple2<>(1, 3), "a")); - data.add(new Tuple2<>(new Tuple2<>(1, 2), "a")); - data.add(new Tuple2<>(new Tuple2<>(2, 1), "a")); - data.add(new Tuple2<>(new Tuple2<>(2, 2), "b")); - data.add(new Tuple2<>(new Tuple2<>(3, 3), "c")); - data.add(new Tuple2<>(new Tuple2<>(3, 6), "c")); - data.add(new Tuple2<>(new Tuple2<>(4, 9), "c")); - - TupleTypeInfo, String>> type = - new TupleTypeInfo<>( - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), - BasicTypeInfo.STRING_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet, String, Integer>> - getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) { - - List, String, Integer>> data = new ArrayList<>(); - data.add(new Tuple3<>(new Tuple2<>(1, 3), "a", 2)); - data.add(new Tuple3<>(new Tuple2<>(1, 2), "a", 1)); - data.add(new Tuple3<>(new Tuple2<>(2, 1), "a", 3)); - data.add(new Tuple3<>(new Tuple2<>(2, 2), "b", 4)); - data.add(new Tuple3<>(new Tuple2<>(3, 3), "c", 5)); - data.add(new Tuple3<>(new Tuple2<>(3, 6), "c", 6)); - data.add(new Tuple3<>(new Tuple2<>(4, 9), "c", 7)); - - TupleTypeInfo, String, Integer>> type = - new TupleTypeInfo<>( - new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet> getTuple2WithByteArrayDataSet( - ExecutionEnvironment env) { - List> data = new ArrayList<>(); - data.add(new Tuple2<>(new byte[] {0, 4}, 1)); - data.add(new Tuple2<>(new byte[] {2, 0}, 1)); - data.add(new Tuple2<>(new byte[] {2, 0, 4}, 4)); - data.add(new Tuple2<>(new byte[] {2, 1}, 3)); - data.add(new Tuple2<>(new byte[] {0}, 0)); - data.add(new Tuple2<>(new byte[] {2, 0}, 1)); - - TupleTypeInfo> type = - new TupleTypeInfo<>( - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet getStringDataSet(ExecutionEnvironment env) { - - List data = new ArrayList<>(); - data.add("Hi"); - data.add("Hello"); - data.add("Hello world"); - data.add("Hello world, how are you?"); - data.add("I am fine."); - data.add("Luke Skywalker"); - data.add("Random comment"); - data.add("LOL"); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet getIntegerDataSet(ExecutionEnvironment env) { - - List data = new ArrayList<>(); - data.add(1); - data.add(2); - data.add(2); - data.add(3); - data.add(3); - data.add(3); - data.add(4); - data.add(4); - data.add(4); - data.add(4); - data.add(5); - data.add(5); - data.add(5); - data.add(5); - data.add(5); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet getCustomTypeDataSet(ExecutionEnvironment env) { - - List data = new ArrayList<>(); - data.add(new CustomType(1, 0L, "Hi")); - data.add(new CustomType(2, 1L, "Hello")); - data.add(new CustomType(2, 2L, "Hello world")); - data.add(new CustomType(3, 3L, "Hello world, how are you?")); - data.add(new CustomType(3, 4L, "I am fine.")); - data.add(new CustomType(3, 5L, "Luke Skywalker")); - data.add(new CustomType(4, 6L, "Comment#1")); - data.add(new CustomType(4, 7L, "Comment#2")); - data.add(new CustomType(4, 8L, "Comment#3")); - data.add(new CustomType(4, 9L, "Comment#4")); - data.add(new CustomType(5, 10L, "Comment#5")); - data.add(new CustomType(5, 11L, "Comment#6")); - data.add(new CustomType(5, 12L, "Comment#7")); - data.add(new CustomType(5, 13L, "Comment#8")); - data.add(new CustomType(5, 14L, "Comment#9")); - data.add(new CustomType(6, 15L, "Comment#10")); - data.add(new CustomType(6, 16L, "Comment#11")); - data.add(new CustomType(6, 17L, "Comment#12")); - data.add(new CustomType(6, 18L, "Comment#13")); - data.add(new CustomType(6, 19L, "Comment#14")); - data.add(new CustomType(6, 20L, "Comment#15")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet getSmallCustomTypeDataSet(ExecutionEnvironment env) { - - List data = new ArrayList<>(); - data.add(new CustomType(1, 0L, "Hi")); - data.add(new CustomType(2, 1L, "Hello")); - data.add(new CustomType(2, 2L, "Hello world")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - /** POJO. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public int myInt; - public long myLong; - public String myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = i; - myLong = l; - myString = s; - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } - - private static class CustomTypeComparator implements Comparator { - @Override - public int compare(CustomType o1, CustomType o2) { - int diff = o1.myInt - o2.myInt; - if (diff != 0) { - return diff; - } - diff = (int) (o1.myLong - o2.myLong); - return diff != 0 ? diff : o1.myString.compareTo(o2.myString); - } - } - - public static DataSet> - getSmallTuplebasedDataSet(ExecutionEnvironment env) { - List> data = - new ArrayList<>(); - data.add(new Tuple7<>(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new Tuple7<>(2, "Second", 20, 200, 2000L, "Two", 20000L)); - data.add(new Tuple7<>(3, "Third", 30, 300, 3000L, "Three", 30000L)); - return env.fromCollection(data); - } - - public static DataSet> - getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) { - List> data = - new ArrayList<>(); - data.add(new Tuple7<>(10000L, 10, 100, 1000L, "One", 1, "First")); - data.add(new Tuple7<>(20000L, 20, 200, 2000L, "Two", 2, "Second")); - data.add(new Tuple7<>(30000L, 30, 300, 3000L, "Three", 3, "Third")); - - return env.fromCollection(data); - } - - public static DataSet getSmallPojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add( - new POJO( - 1 /*number*/, - "First" /*str*/, - 10 /*f0*/, - 100 /*f1.myInt*/, - 1000L /*f1.myLong*/, - "One" /*f1.myString*/, - 10000L /*nestedPojo.longNumber*/)); - data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L)); - data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); - return env.fromCollection(data); - } - - public static DataSet getDuplicatePojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L)); - data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x - data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); - return env.fromCollection(data); - } - - public static DataSet getMixedPojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x - data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L)); - data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L)); - data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L)); - data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L)); - data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L)); - data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x - data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L)); - return env.fromCollection(data); - } - - /** POJO. */ - public static class POJO { - public int number; - public String str; - public Tuple2 nestedTupleWithCustom; - public NestedPojo nestedPojo; - public transient Long ignoreMe; - - public POJO(int i0, String s0, int i1, int i2, long l0, String s1, long l1) { - this.number = i0; - this.str = s0; - this.nestedTupleWithCustom = new Tuple2<>(i1, new CustomType(i2, l0, s1)); - this.nestedPojo = new NestedPojo(); - this.nestedPojo.longNumber = l1; - } - - public POJO() {} - - @Override - public String toString() { - return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber; - } - } - - /** Nested POJO. */ - public static class NestedPojo { - public static Object ignoreMe; - public long longNumber; - - public NestedPojo() {} - } - - public static DataSet getCrazyNestedDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new CrazyNested("aa")); - data.add(new CrazyNested("bb")); - data.add(new CrazyNested("bb")); - data.add(new CrazyNested("cc")); - data.add(new CrazyNested("cc")); - data.add(new CrazyNested("cc")); - return env.fromCollection(data); - } - - /** Deeply nested POJO. */ - public static class CrazyNested { - public CrazyNestedL1 nestLvl1; - public Long something; // test proper null-value handling - - public CrazyNested() {} - - public CrazyNested( - String set, - String second, - long s) { // additional CTor to set all fields to non-null values - this(set); - something = s; - nestLvl1.a = second; - } - - public CrazyNested(String set) { - nestLvl1 = new CrazyNestedL1(); - nestLvl1.nestLvl2 = new CrazyNestedL2(); - nestLvl1.nestLvl2.nestLvl3 = new CrazyNestedL3(); - nestLvl1.nestLvl2.nestLvl3.nestLvl4 = new CrazyNestedL4(); - nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal = set; - } - } - - /** Nested POJO level 2. */ - public static class CrazyNestedL1 { - public String a; - public int b; - public CrazyNestedL2 nestLvl2; - } - - /** Nested POJO level 3. */ - public static class CrazyNestedL2 { - public CrazyNestedL3 nestLvl3; - } - - /** Nested POJO level 4. */ - public static class CrazyNestedL3 { - public CrazyNestedL4 nestLvl4; - } - - /** Nested POJO level 5. */ - public static class CrazyNestedL4 { - public String f1nal; - } - - // Copied from TypeExtractorTest - private static class FromTuple extends Tuple3 { - private static final long serialVersionUID = 1L; - public int special; - } - - /** Pojo extending from tuple WITH custom fields. */ - public static class FromTupleWithCTor extends FromTuple { - - private static final long serialVersionUID = 1L; - - public FromTupleWithCTor() {} - - public FromTupleWithCTor(int special, long tupleField) { - this.special = special; - this.setField(tupleField, 2); - } - } - - public static DataSet getPojoExtendingFromTuple(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new FromTupleWithCTor(1, 10L)); // 3x - data.add(new FromTupleWithCTor(1, 10L)); - data.add(new FromTupleWithCTor(1, 10L)); - data.add(new FromTupleWithCTor(2, 20L)); // 2x - data.add(new FromTupleWithCTor(2, 20L)); - return env.fromCollection(data); - } - - /** POJO with Tuple and Writable. */ - public static class PojoContainingTupleAndWritable { - public int someInt; - public String someString; - public IntWritable hadoopFan; - public Tuple2 theTuple; - - public PojoContainingTupleAndWritable() {} - - public PojoContainingTupleAndWritable(int i, long l1, long l2) { - hadoopFan = new IntWritable(i); - someInt = i; - theTuple = new Tuple2<>(l1, l2); - } - } - - public static DataSet getPojoContainingTupleAndWritable( - ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - return env.fromCollection(data); - } - - public static DataSet - getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x - data.add(new PojoContainingTupleAndWritable(2, 20L, 201L)); - data.add(new PojoContainingTupleAndWritable(2, 30L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 30L, 600L)); - data.add(new PojoContainingTupleAndWritable(2, 30L, 400L)); - return env.fromCollection(data); - } - - public static DataSet> getTupleContainingPojos( - ExecutionEnvironment env) { - List> data = new ArrayList<>(); - data.add( - new Tuple3<>( - 1, - new CrazyNested("one", "uno", 1L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x - data.add( - new Tuple3<>( - 1, - new CrazyNested("one", "uno", 1L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); - data.add( - new Tuple3<>( - 1, - new CrazyNested("one", "uno", 1L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); - // POJO is not initialized according to the first two fields. - data.add( - new Tuple3<>( - 2, - new CrazyNested("two", "duo", 2L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x - return env.fromCollection(data); - } - - /** POJO. */ - public static class Pojo1 { - public String a; - public String b; - - public Pojo1() {} - - public Pojo1(String a, String b) { - this.a = a; - this.b = b; - } - } - - /** Another POJO. */ - public static class Pojo2 { - public String a2; - public String b2; - } - - /** Nested POJO. */ - public static class PojoWithMultiplePojos { - public Pojo1 p1; - public Pojo2 p2; - public Integer i0; - - public PojoWithMultiplePojos() {} - - public PojoWithMultiplePojos(String a, String b, String a1, String b1, Integer i0) { - p1 = new Pojo1(); - p1.a = a; - p1.b = b; - p2 = new Pojo2(); - p2.a2 = a1; - p2.b2 = b1; - this.i0 = i0; - } - } - - public static DataSet getPojoWithMultiplePojos( - ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1)); - data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2)); - data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2)); - data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2)); - data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3)); - data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3)); - return env.fromCollection(data); - } - - /** Custom enum. */ - public enum Category { - CAT_A, - CAT_B - } - - /** POJO with Date and enum. */ - public static class PojoWithDateAndEnum { - public String group; - public Date date; - public Category cat; - } - - public static DataSet getPojoWithDateAndEnum(ExecutionEnvironment env) { - List data = new ArrayList<>(); - - PojoWithDateAndEnum one = new PojoWithDateAndEnum(); - one.group = "a"; - one.date = new Date(666); - one.cat = Category.CAT_A; - data.add(one); - - PojoWithDateAndEnum two = new PojoWithDateAndEnum(); - two.group = "a"; - two.date = new Date(666); - two.cat = Category.CAT_A; - data.add(two); - - PojoWithDateAndEnum three = new PojoWithDateAndEnum(); - three.group = "b"; - three.date = new Date(666); - three.cat = Category.CAT_B; - data.add(three); - - return env.fromCollection(data); - } - - /** POJO with collection. */ - public static class PojoWithCollection { - public List pojos; - public int key; - public java.sql.Date sqlDate; - public BigInteger bigInt; - public BigDecimal bigDecimalKeepItNull; - public BigInt scalaBigInt; - public List mixed; - - @Override - public String toString() { - return "PojoWithCollection{" - + "pojos.size()=" - + pojos.size() - + ", key=" - + key - + ", sqlDate=" - + sqlDate - + ", bigInt=" - + bigInt - + ", bigDecimalKeepItNull=" - + bigDecimalKeepItNull - + ", scalaBigInt=" - + scalaBigInt - + ", mixed=" - + mixed - + '}'; - } - } - - /** POJO with generic collection. */ - public static class PojoWithCollectionGeneric { - public List pojos; - public int key; - public java.sql.Date sqlDate; - public BigInteger bigInt; - public BigDecimal bigDecimalKeepItNull; - public BigInt scalaBigInt; - public List mixed; - private PojoWithDateAndEnum makeMeGeneric; - - @Override - public String toString() { - return "PojoWithCollection{" - + "pojos.size()=" - + pojos.size() - + ", key=" - + key - + ", sqlDate=" - + sqlDate - + ", bigInt=" - + bigInt - + ", bigDecimalKeepItNull=" - + bigDecimalKeepItNull - + ", scalaBigInt=" - + scalaBigInt - + ", mixed=" - + mixed - + '}'; - } - } - - public static DataSet getPojoWithCollection(ExecutionEnvironment env) { - List data = new ArrayList<>(); - - List pojosList1 = new ArrayList<>(); - pojosList1.add(new Pojo1("a", "aa")); - pojosList1.add(new Pojo1("b", "bb")); - - List pojosList2 = new ArrayList<>(); - pojosList2.add(new Pojo1("a2", "aa2")); - pojosList2.add(new Pojo1("b2", "bb2")); - - PojoWithCollection pwc1 = new PojoWithCollection(); - pwc1.pojos = pojosList1; - pwc1.key = 0; - pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN); - pwc1.scalaBigInt = BigInt.int2bigInt(10); - pwc1.bigDecimalKeepItNull = null; - - // use calendar to make it stable across time zones - GregorianCalendar gcl1 = new GregorianCalendar(2033, 4, 18); - pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis()); - pwc1.mixed = new ArrayList<>(); - Map map = new HashMap<>(); - map.put("someKey", 1); // map.put("anotherKey", 2); map.put("third", 3); - pwc1.mixed.add(map); - pwc1.mixed.add(new File("/this/is/wrong")); - pwc1.mixed.add("uhlala"); - - PojoWithCollection pwc2 = new PojoWithCollection(); - pwc2.pojos = pojosList2; - pwc2.key = 0; - pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN); - pwc2.scalaBigInt = BigInt.int2bigInt(31104000); - pwc2.bigDecimalKeepItNull = null; - - GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3); - pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976 - - data.add(pwc1); - data.add(pwc2); - - return env.fromCollection(data); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataStreams.java b/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataStreams.java new file mode 100644 index 0000000000000..7913abcb76461 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataStreams.java @@ -0,0 +1,247 @@ +/* + * 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.operators.util; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.List; + +import scala.math.BigInt; + +/** + * ####################################################################################################### + * BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA. IF YOU MODIFY THE DATA MAKE SURE YOU CHECK + * THAT ALL TESTS ARE STILL WORKING! + * ####################################################################################################### + */ +public class CollectionDataStreams { + + public static DataStreamSource> get3TupleDataSet( + StreamExecutionEnvironment env) { + + List> data = new ArrayList<>(); + data.add(new Tuple3<>(1, 1L, "Hi")); + data.add(new Tuple3<>(2, 2L, "Hello")); + data.add(new Tuple3<>(3, 2L, "Hello world")); + data.add(new Tuple3<>(4, 3L, "Hello world, how are you?")); + data.add(new Tuple3<>(5, 3L, "I am fine.")); + data.add(new Tuple3<>(6, 3L, "Luke Skywalker")); + data.add(new Tuple3<>(7, 4L, "Comment#1")); + data.add(new Tuple3<>(8, 4L, "Comment#2")); + data.add(new Tuple3<>(9, 4L, "Comment#3")); + data.add(new Tuple3<>(10, 4L, "Comment#4")); + data.add(new Tuple3<>(11, 5L, "Comment#5")); + data.add(new Tuple3<>(12, 5L, "Comment#6")); + data.add(new Tuple3<>(13, 5L, "Comment#7")); + data.add(new Tuple3<>(14, 5L, "Comment#8")); + data.add(new Tuple3<>(15, 5L, "Comment#9")); + data.add(new Tuple3<>(16, 6L, "Comment#10")); + data.add(new Tuple3<>(17, 6L, "Comment#11")); + data.add(new Tuple3<>(18, 6L, "Comment#12")); + data.add(new Tuple3<>(19, 6L, "Comment#13")); + data.add(new Tuple3<>(20, 6L, "Comment#14")); + data.add(new Tuple3<>(21, 6L, "Comment#15")); + + Collections.shuffle(data); + + return env.fromData(data); + } + + public static DataStreamSource> getSmall3TupleDataSet( + StreamExecutionEnvironment env) { + + List> data = new ArrayList<>(); + data.add(new Tuple3<>(1, 1L, "Hi")); + data.add(new Tuple3<>(2, 2L, "Hello")); + data.add(new Tuple3<>(3, 2L, "Hello world")); + + Collections.shuffle(data); + + return env.fromData(data); + } + + /** POJO. */ + public static class CustomType implements Serializable { + + private static final long serialVersionUID = 1L; + + public int myInt; + public long myLong; + public String myString; + + public CustomType() {} + + public CustomType(int i, long l, String s) { + myInt = i; + myLong = l; + myString = s; + } + + @Override + public String toString() { + return myInt + "," + myLong + "," + myString; + } + } + + /** POJO. */ + public static class POJO { + public int number; + public String str; + public Tuple2 nestedTupleWithCustom; + public NestedPojo nestedPojo; + public transient Long ignoreMe; + + public POJO(int i0, String s0, int i1, int i2, long l0, String s1, long l1) { + this.number = i0; + this.str = s0; + this.nestedTupleWithCustom = new Tuple2<>(i1, new CustomType(i2, l0, s1)); + this.nestedPojo = new NestedPojo(); + this.nestedPojo.longNumber = l1; + } + + public POJO() {} + + @Override + public String toString() { + return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber; + } + } + + /** Nested POJO. */ + public static class NestedPojo { + public static Object ignoreMe; + public long longNumber; + + public NestedPojo() {} + } + + /** Deeply nested POJO. */ + public static class CrazyNested { + public CrazyNestedL1 nestLvl1; + public Long something; // test proper null-value handling + + public CrazyNested() {} + + public CrazyNested( + String set, + String second, + long s) { // additional CTor to set all fields to non-null values + this(set); + something = s; + nestLvl1.a = second; + } + + public CrazyNested(String set) { + nestLvl1 = new CrazyNestedL1(); + nestLvl1.nestLvl2 = new CrazyNestedL2(); + nestLvl1.nestLvl2.nestLvl3 = new CrazyNestedL3(); + nestLvl1.nestLvl2.nestLvl3.nestLvl4 = new CrazyNestedL4(); + nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal = set; + } + } + + /** Nested POJO level 2. */ + public static class CrazyNestedL1 { + public String a; + public int b; + public CrazyNestedL2 nestLvl2; + } + + /** Nested POJO level 3. */ + public static class CrazyNestedL2 { + public CrazyNestedL3 nestLvl3; + } + + /** Nested POJO level 4. */ + public static class CrazyNestedL3 { + public CrazyNestedL4 nestLvl4; + } + + /** Nested POJO level 5. */ + public static class CrazyNestedL4 { + public String f1nal; + } + + /** POJO. */ + public static class Pojo1 { + public String a; + public String b; + + public Pojo1() {} + + public Pojo1(String a, String b) { + this.a = a; + this.b = b; + } + } + + /** Custom enum. */ + public enum Category { + CAT_A, + CAT_B + } + + /** POJO with Date and enum. */ + public static class PojoWithDateAndEnum { + public String group; + public Date date; + public Category cat; + } + + /** POJO with generic collection. */ + public static class PojoWithCollectionGeneric { + public List pojos; + public int key; + public java.sql.Date sqlDate; + public BigInteger bigInt; + public BigDecimal bigDecimalKeepItNull; + public BigInt scalaBigInt; + public List mixed; + private PojoWithDateAndEnum makeMeGeneric; + + @Override + public String toString() { + return "PojoWithCollection{" + + "pojos.size()=" + + pojos.size() + + ", key=" + + key + + ", sqlDate=" + + sqlDate + + ", bigInt=" + + bigInt + + ", bigDecimalKeepItNull=" + + bigDecimalKeepItNull + + ", scalaBigInt=" + + scalaBigInt + + ", mixed=" + + mixed + + '}'; + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java deleted file mode 100644 index ff27c7510533b..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java +++ /dev/null @@ -1,1011 +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.operators.util; - -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.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.types.IntValue; -import org.apache.flink.types.LongValue; -import org.apache.flink.types.StringValue; - -import org.apache.hadoop.io.IntWritable; - -import java.io.File; -import java.io.Serializable; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.GregorianCalendar; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import scala.math.BigInt; - -/** - * ####################################################################################################### - * BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA. IF YOU MODIFY THE DATA MAKE SURE YOU CHECK - * THAT ALL TESTS ARE STILL WORKING! - * ####################################################################################################### - */ -public class ValueCollectionDataSets { - - public static DataSet> get3TupleDataSet( - ExecutionEnvironment env) { - List> data = new ArrayList<>(); - - data.add(new Tuple3<>(new IntValue(1), new LongValue(1L), new StringValue("Hi"))); - data.add(new Tuple3<>(new IntValue(2), new LongValue(2L), new StringValue("Hello"))); - data.add(new Tuple3<>(new IntValue(3), new LongValue(2L), new StringValue("Hello world"))); - data.add( - new Tuple3<>( - new IntValue(4), - new LongValue(3L), - new StringValue("Hello world, how are you?"))); - data.add(new Tuple3<>(new IntValue(5), new LongValue(3L), new StringValue("I am fine."))); - data.add( - new Tuple3<>( - new IntValue(6), new LongValue(3L), new StringValue("Luke Skywalker"))); - data.add(new Tuple3<>(new IntValue(7), new LongValue(4L), new StringValue("Comment#1"))); - data.add(new Tuple3<>(new IntValue(8), new LongValue(4L), new StringValue("Comment#2"))); - data.add(new Tuple3<>(new IntValue(9), new LongValue(4L), new StringValue("Comment#3"))); - data.add(new Tuple3<>(new IntValue(10), new LongValue(4L), new StringValue("Comment#4"))); - data.add(new Tuple3<>(new IntValue(11), new LongValue(5L), new StringValue("Comment#5"))); - data.add(new Tuple3<>(new IntValue(12), new LongValue(5L), new StringValue("Comment#6"))); - data.add(new Tuple3<>(new IntValue(13), new LongValue(5L), new StringValue("Comment#7"))); - data.add(new Tuple3<>(new IntValue(14), new LongValue(5L), new StringValue("Comment#8"))); - data.add(new Tuple3<>(new IntValue(15), new LongValue(5L), new StringValue("Comment#9"))); - data.add(new Tuple3<>(new IntValue(16), new LongValue(6L), new StringValue("Comment#10"))); - data.add(new Tuple3<>(new IntValue(17), new LongValue(6L), new StringValue("Comment#11"))); - data.add(new Tuple3<>(new IntValue(18), new LongValue(6L), new StringValue("Comment#12"))); - data.add(new Tuple3<>(new IntValue(19), new LongValue(6L), new StringValue("Comment#13"))); - data.add(new Tuple3<>(new IntValue(20), new LongValue(6L), new StringValue("Comment#14"))); - data.add(new Tuple3<>(new IntValue(21), new LongValue(6L), new StringValue("Comment#15"))); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet> getSmall3TupleDataSet( - ExecutionEnvironment env) { - List> data = new ArrayList<>(); - - data.add(new Tuple3<>(new IntValue(1), new LongValue(1L), new StringValue("Hi"))); - data.add(new Tuple3<>(new IntValue(2), new LongValue(2L), new StringValue("Hello"))); - data.add(new Tuple3<>(new IntValue(3), new LongValue(2L), new StringValue("Hello world"))); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet> - get5TupleDataSet(ExecutionEnvironment env) { - List> data = - new ArrayList<>(); - - data.add( - new Tuple5<>( - new IntValue(1), - new LongValue(1L), - new IntValue(0), - new StringValue("Hallo"), - new LongValue(1L))); - data.add( - new Tuple5<>( - new IntValue(2), - new LongValue(2L), - new IntValue(1), - new StringValue("Hallo Welt"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(2), - new LongValue(3L), - new IntValue(2), - new StringValue("Hallo Welt wie"), - new LongValue(1L))); - data.add( - new Tuple5<>( - new IntValue(3), - new LongValue(4L), - new IntValue(3), - new StringValue("Hallo Welt wie gehts?"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(3), - new LongValue(5L), - new IntValue(4), - new StringValue("ABC"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(3), - new LongValue(6L), - new IntValue(5), - new StringValue("BCD"), - new LongValue(3L))); - data.add( - new Tuple5<>( - new IntValue(4), - new LongValue(7L), - new IntValue(6), - new StringValue("CDE"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(4), - new LongValue(8L), - new IntValue(7), - new StringValue("DEF"), - new LongValue(1L))); - data.add( - new Tuple5<>( - new IntValue(4), - new LongValue(9L), - new IntValue(8), - new StringValue("EFG"), - new LongValue(1L))); - data.add( - new Tuple5<>( - new IntValue(4), - new LongValue(10L), - new IntValue(9), - new StringValue("FGH"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(5), - new LongValue(11L), - new IntValue(10), - new StringValue("GHI"), - new LongValue(1L))); - data.add( - new Tuple5<>( - new IntValue(5), - new LongValue(12L), - new IntValue(11), - new StringValue("HIJ"), - new LongValue(3L))); - data.add( - new Tuple5<>( - new IntValue(5), - new LongValue(13L), - new IntValue(12), - new StringValue("IJK"), - new LongValue(3L))); - data.add( - new Tuple5<>( - new IntValue(5), - new LongValue(14L), - new IntValue(13), - new StringValue("JKL"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(5), - new LongValue(15L), - new IntValue(14), - new StringValue("KLM"), - new LongValue(2L))); - - Collections.shuffle(data); - - TupleTypeInfo> type = - new TupleTypeInfo<>( - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.LONG_VALUE_TYPE_INFO, - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.STRING_VALUE_TYPE_INFO, - ValueTypeInfo.LONG_VALUE_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet> - getSmall5TupleDataSet(ExecutionEnvironment env) { - List> data = - new ArrayList<>(); - - data.add( - new Tuple5<>( - new IntValue(1), - new LongValue(1L), - new IntValue(0), - new StringValue("Hallo"), - new LongValue(1L))); - data.add( - new Tuple5<>( - new IntValue(2), - new LongValue(2L), - new IntValue(1), - new StringValue("Hallo Welt"), - new LongValue(2L))); - data.add( - new Tuple5<>( - new IntValue(2), - new LongValue(3L), - new IntValue(2), - new StringValue("Hallo Welt wie"), - new LongValue(1L))); - - Collections.shuffle(data); - - TupleTypeInfo> type = - new TupleTypeInfo<>( - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.LONG_VALUE_TYPE_INFO, - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.STRING_VALUE_TYPE_INFO, - ValueTypeInfo.LONG_VALUE_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet, StringValue>> - getSmallNestedTupleDataSet(ExecutionEnvironment env) { - List, StringValue>> data = new ArrayList<>(); - - data.add( - new Tuple2<>( - new Tuple2<>(new IntValue(1), new IntValue(1)), new StringValue("one"))); - data.add( - new Tuple2<>( - new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("two"))); - data.add( - new Tuple2<>( - new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("three"))); - - TupleTypeInfo, StringValue>> type = - new TupleTypeInfo<>( - new TupleTypeInfo>( - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.INT_VALUE_TYPE_INFO), - ValueTypeInfo.STRING_VALUE_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet, StringValue>> - getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) { - List, StringValue>> data = new ArrayList<>(); - - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(3)), new StringValue("a"))); - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(2)), new StringValue("a"))); - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(1)), new StringValue("a"))); - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("b"))); - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("c"))); - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(6)), new StringValue("c"))); - data.add( - new Tuple2<>(new Tuple2<>(new IntValue(4), new IntValue(9)), new StringValue("c"))); - - TupleTypeInfo, StringValue>> type = - new TupleTypeInfo<>( - new TupleTypeInfo>( - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.INT_VALUE_TYPE_INFO), - ValueTypeInfo.STRING_VALUE_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet, StringValue, IntValue>> - getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) { - List, StringValue, IntValue>> data = new ArrayList<>(); - - data.add( - new Tuple3<>( - new Tuple2(new IntValue(1), new IntValue(3)), - new StringValue("a"), - new IntValue(2))); - data.add( - new Tuple3<>( - new Tuple2(new IntValue(1), new IntValue(2)), - new StringValue("a"), - new IntValue(1))); - data.add( - new Tuple3<>( - new Tuple2(new IntValue(2), new IntValue(1)), - new StringValue("a"), - new IntValue(3))); - data.add( - new Tuple3<>( - new Tuple2(new IntValue(2), new IntValue(2)), - new StringValue("b"), - new IntValue(4))); - data.add( - new Tuple3<>( - new Tuple2(new IntValue(3), new IntValue(3)), - new StringValue("c"), - new IntValue(5))); - data.add( - new Tuple3<>( - new Tuple2(new IntValue(3), new IntValue(6)), - new StringValue("c"), - new IntValue(6))); - data.add( - new Tuple3<>( - new Tuple2(new IntValue(4), new IntValue(9)), - new StringValue("c"), - new IntValue(7))); - - TupleTypeInfo, StringValue, IntValue>> type = - new TupleTypeInfo<>( - new TupleTypeInfo>( - ValueTypeInfo.INT_VALUE_TYPE_INFO, - ValueTypeInfo.INT_VALUE_TYPE_INFO), - ValueTypeInfo.STRING_VALUE_TYPE_INFO, - ValueTypeInfo.INT_VALUE_TYPE_INFO); - - return env.fromCollection(data, type); - } - - public static DataSet getStringDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - - data.add(new StringValue("Hi")); - data.add(new StringValue("Hello")); - data.add(new StringValue("Hello world")); - data.add(new StringValue("Hello world, how are you?")); - data.add(new StringValue("I am fine.")); - data.add(new StringValue("Luke Skywalker")); - data.add(new StringValue("Random comment")); - data.add(new StringValue("LOL")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet getIntDataSet(ExecutionEnvironment env) { - List data = new ArrayList<>(); - - data.add(new IntValue(1)); - data.add(new IntValue(2)); - data.add(new IntValue(2)); - data.add(new IntValue(3)); - data.add(new IntValue(3)); - data.add(new IntValue(3)); - data.add(new IntValue(4)); - data.add(new IntValue(4)); - data.add(new IntValue(4)); - data.add(new IntValue(4)); - data.add(new IntValue(5)); - data.add(new IntValue(5)); - data.add(new IntValue(5)); - data.add(new IntValue(5)); - data.add(new IntValue(5)); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet getCustomTypeDataSet(ExecutionEnvironment env) { - List data = new ArrayList(); - - data.add(new CustomType(1, 0L, "Hi")); - data.add(new CustomType(2, 1L, "Hello")); - data.add(new CustomType(2, 2L, "Hello world")); - data.add(new CustomType(3, 3L, "Hello world, how are you?")); - data.add(new CustomType(3, 4L, "I am fine.")); - data.add(new CustomType(3, 5L, "Luke Skywalker")); - data.add(new CustomType(4, 6L, "Comment#1")); - data.add(new CustomType(4, 7L, "Comment#2")); - data.add(new CustomType(4, 8L, "Comment#3")); - data.add(new CustomType(4, 9L, "Comment#4")); - data.add(new CustomType(5, 10L, "Comment#5")); - data.add(new CustomType(5, 11L, "Comment#6")); - data.add(new CustomType(5, 12L, "Comment#7")); - data.add(new CustomType(5, 13L, "Comment#8")); - data.add(new CustomType(5, 14L, "Comment#9")); - data.add(new CustomType(6, 15L, "Comment#10")); - data.add(new CustomType(6, 16L, "Comment#11")); - data.add(new CustomType(6, 17L, "Comment#12")); - data.add(new CustomType(6, 18L, "Comment#13")); - data.add(new CustomType(6, 19L, "Comment#14")); - data.add(new CustomType(6, 20L, "Comment#15")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - public static DataSet getSmallCustomTypeDataSet(ExecutionEnvironment env) { - List data = new ArrayList(); - - data.add(new CustomType(1, 0L, "Hi")); - data.add(new CustomType(2, 1L, "Hello")); - data.add(new CustomType(2, 2L, "Hello world")); - - Collections.shuffle(data); - - return env.fromCollection(data); - } - - /** POJO. */ - public static class CustomType implements Serializable { - - private static final long serialVersionUID = 1L; - - public IntValue myInt; - public LongValue myLong; - public StringValue myString; - - public CustomType() {} - - public CustomType(int i, long l, String s) { - myInt = new IntValue(i); - myLong = new LongValue(l); - myString = new StringValue(s); - } - - @Override - public String toString() { - return myInt + "," + myLong + "," + myString; - } - } - - private static class CustomTypeComparator implements Comparator { - - @Override - public int compare(CustomType o1, CustomType o2) { - int diff = o1.myInt.getValue() - o2.myInt.getValue(); - if (diff != 0) { - return diff; - } - diff = (int) (o1.myLong.getValue() - o2.myLong.getValue()); - return diff != 0 ? diff : o1.myString.getValue().compareTo(o2.myString.getValue()); - } - } - - private static DataSet< - Tuple7< - IntValue, - StringValue, - IntValue, - IntValue, - LongValue, - StringValue, - LongValue>> - getSmallTuplebasedDataSet(ExecutionEnvironment env) { - List> - data = new ArrayList<>(); - - data.add( - new Tuple7<>( - new IntValue(1), - new StringValue("First"), - new IntValue(10), - new IntValue(100), - new LongValue(1000L), - new StringValue("One"), - new LongValue(10000L))); - data.add( - new Tuple7<>( - new IntValue(2), - new StringValue("Second"), - new IntValue(20), - new IntValue(200), - new LongValue(2000L), - new StringValue("Two"), - new LongValue(20000L))); - data.add( - new Tuple7<>( - new IntValue(3), - new StringValue("Third"), - new IntValue(30), - new IntValue(300), - new LongValue(3000L), - new StringValue("Three"), - new LongValue(30000L))); - - return env.fromCollection(data); - } - - private static DataSet< - Tuple7< - LongValue, - IntValue, - IntValue, - LongValue, - StringValue, - IntValue, - StringValue>> - getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) { - List> - data = new ArrayList<>(); - - data.add( - new Tuple7<>( - new LongValue(10000L), - new IntValue(10), - new IntValue(100), - new LongValue(1000L), - new StringValue("One"), - new IntValue(1), - new StringValue("First"))); - data.add( - new Tuple7<>( - new LongValue(20000L), - new IntValue(20), - new IntValue(200), - new LongValue(2000L), - new StringValue("Two"), - new IntValue(2), - new StringValue("Second"))); - data.add( - new Tuple7<>( - new LongValue(30000L), - new IntValue(30), - new IntValue(300), - new LongValue(3000L), - new StringValue("Three"), - new IntValue(3), - new StringValue("Third"))); - - return env.fromCollection(data); - } - - private static DataSet getSmallPojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList(); - - data.add( - new POJO( - 1 /*number*/, - "First" /*str*/, - 10 /*f0*/, - 100 /*f1.myInt*/, - 1000L /*f1.myLong*/, - "One" /*f1.myString*/, - 10000L /*nestedPojo.longNumber*/)); - data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L)); - data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); - - return env.fromCollection(data); - } - - private static DataSet getDuplicatePojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList(); - - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); - data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L)); - data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x - data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); - - return env.fromCollection(data); - } - - private static DataSet getMixedPojoDataSet(ExecutionEnvironment env) { - List data = new ArrayList(); - - data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x - data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L)); - data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L)); - data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L)); - data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L)); - data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L)); - data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x - data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L)); - - return env.fromCollection(data); - } - - /** POJO. */ - public static class POJO { - public IntValue number; - public StringValue str; - public Tuple2 nestedTupleWithCustom; - public NestedPojo nestedPojo; - public transient LongValue ignoreMe; - - public POJO(int i0, String s0, int i1, int i2, long l0, String s1, long l1) { - this.number = new IntValue(i0); - this.str = new StringValue(s0); - this.nestedTupleWithCustom = new Tuple2<>(new IntValue(i1), new CustomType(i2, l0, s1)); - this.nestedPojo = new NestedPojo(); - this.nestedPojo.longNumber = new LongValue(l1); - } - - public POJO() {} - - @Override - public String toString() { - return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber; - } - } - - /** Nested POJO. */ - public static class NestedPojo { - public static Object ignoreMe; - public LongValue longNumber; - - public NestedPojo() {} - } - - private static DataSet getCrazyNestedDataSet(ExecutionEnvironment env) { - List data = new ArrayList(); - - data.add(new CrazyNested("aa")); - data.add(new CrazyNested("bb")); - data.add(new CrazyNested("bb")); - data.add(new CrazyNested("cc")); - data.add(new CrazyNested("cc")); - data.add(new CrazyNested("cc")); - - return env.fromCollection(data); - } - - /** Deeply nested POJO. */ - public static class CrazyNested { - public CrazyNestedL1 nestLvl1; - public LongValue something; // test proper null-value handling - - public CrazyNested() {} - - public CrazyNested( - String set, - String second, - long s) { // additional CTor to set all fields to non-null values - this(set); - something = new LongValue(s); - nestLvl1.a = new StringValue(second); - } - - public CrazyNested(String set) { - nestLvl1 = new CrazyNestedL1(); - nestLvl1.nestLvl2 = new CrazyNestedL2(); - nestLvl1.nestLvl2.nestLvl3 = new CrazyNestedL3(); - nestLvl1.nestLvl2.nestLvl3.nestLvl4 = new CrazyNestedL4(); - nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal = new StringValue(set); - } - } - - /** Nested POJO level 2. */ - public static class CrazyNestedL1 { - public StringValue a; - public IntValue b; - public CrazyNestedL2 nestLvl2; - } - - /** Nested POJO level 3. */ - public static class CrazyNestedL2 { - public CrazyNestedL3 nestLvl3; - } - - /** Nested POJO level 4. */ - public static class CrazyNestedL3 { - public CrazyNestedL4 nestLvl4; - } - - /** Nested POJO level 5. */ - public static class CrazyNestedL4 { - public StringValue f1nal; - } - - // Copied from TypeExtractorTest - private static class FromTuple extends Tuple3 { - private static final long serialVersionUID = 1L; - public IntValue special; - } - - private static class FromTupleWithCTor extends FromTuple { - - private static final long serialVersionUID = 1L; - - public FromTupleWithCTor() {} - - public FromTupleWithCTor(int special, long tupleField) { - this.special = new IntValue(special); - this.setField(new LongValue(tupleField), 2); - } - } - - public static DataSet getPojoExtendingFromTuple(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new FromTupleWithCTor(1, 10L)); // 3x - data.add(new FromTupleWithCTor(1, 10L)); - data.add(new FromTupleWithCTor(1, 10L)); - data.add(new FromTupleWithCTor(2, 20L)); // 2x - data.add(new FromTupleWithCTor(2, 20L)); - return env.fromCollection(data); - } - - /** POJO with Tuple and Writable. */ - public static class PojoContainingTupleAndWritable { - public IntValue someInt; - public StringValue someString; - public IntWritable hadoopFan; - public Tuple2 theTuple; - - public PojoContainingTupleAndWritable() {} - - public PojoContainingTupleAndWritable(int i, long l1, long l2) { - hadoopFan = new IntWritable(i); - someInt = new IntValue(i); - theTuple = new Tuple2<>(new LongValue(l1), new LongValue(l2)); - } - } - - public static DataSet getPojoContainingTupleAndWritable( - ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); - return env.fromCollection(data); - } - - public static DataSet - getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x - data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x - data.add(new PojoContainingTupleAndWritable(2, 20L, 201L)); - data.add(new PojoContainingTupleAndWritable(2, 30L, 200L)); - data.add(new PojoContainingTupleAndWritable(2, 30L, 600L)); - data.add(new PojoContainingTupleAndWritable(2, 30L, 400L)); - return env.fromCollection(data); - } - - public static DataSet> getTupleContainingPojos( - ExecutionEnvironment env) { - List> data = new ArrayList<>(); - data.add( - new Tuple3( - new IntValue(1), - new CrazyNested("one", "uno", 1L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x - data.add( - new Tuple3( - new IntValue(1), - new CrazyNested("one", "uno", 1L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); - data.add( - new Tuple3( - new IntValue(1), - new CrazyNested("one", "uno", 1L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); - // POJO is not initialized according to the first two fields. - data.add( - new Tuple3( - new IntValue(2), - new CrazyNested("two", "duo", 2L), - new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x - return env.fromCollection(data); - } - - /** POJO. */ - public static class Pojo1 { - public StringValue a; - public StringValue b; - - public Pojo1() {} - - public Pojo1(String a, String b) { - this.a = new StringValue(a); - this.b = new StringValue(b); - } - } - - /** POJO. */ - public static class Pojo2 { - public StringValue a2; - public StringValue b2; - } - - /** Nested POJO. */ - public static class PojoWithMultiplePojos { - public Pojo1 p1; - public Pojo2 p2; - public IntValue i0; - - public PojoWithMultiplePojos() {} - - public PojoWithMultiplePojos(String a, String b, String a1, String b1, int i0) { - p1 = new Pojo1(); - p1.a = new StringValue(a); - p1.b = new StringValue(b); - p2 = new Pojo2(); - p2.a2 = new StringValue(a1); - p2.b2 = new StringValue(b1); - this.i0 = new IntValue(i0); - } - } - - public static DataSet getPojoWithMultiplePojos( - ExecutionEnvironment env) { - List data = new ArrayList<>(); - data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1)); - data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2)); - data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2)); - data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2)); - data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3)); - data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3)); - return env.fromCollection(data); - } - - /** Custom enum. */ - public enum Category { - CAT_A, - CAT_B; - } - - /** POJO with Data and enum. */ - public static class PojoWithDateAndEnum { - public StringValue group; - public Date date; - public Category cat; - } - - public static DataSet getPojoWithDateAndEnum(ExecutionEnvironment env) { - List data = new ArrayList(); - - PojoWithDateAndEnum one = new PojoWithDateAndEnum(); - one.group = new StringValue("a"); - one.date = new Date(666); - one.cat = Category.CAT_A; - data.add(one); - - PojoWithDateAndEnum two = new PojoWithDateAndEnum(); - two.group = new StringValue("a"); - two.date = new Date(666); - two.cat = Category.CAT_A; - data.add(two); - - PojoWithDateAndEnum three = new PojoWithDateAndEnum(); - three.group = new StringValue("b"); - three.date = new Date(666); - three.cat = Category.CAT_B; - data.add(three); - - return env.fromCollection(data); - } - - /** POJO with collection. */ - public static class PojoWithCollection { - public List pojos; - public IntValue key; - public java.sql.Date sqlDate; - public BigInteger bigInt; - public BigDecimal bigDecimalKeepItNull; - public BigInt scalaBigInt; - public List mixed; - - @Override - public String toString() { - return "PojoWithCollection{" - + "pojos.size()=" - + pojos.size() - + ", key=" - + key - + ", sqlDate=" - + sqlDate - + ", bigInt=" - + bigInt - + ", bigDecimalKeepItNull=" - + bigDecimalKeepItNull - + ", scalaBigInt=" - + scalaBigInt - + ", mixed=" - + mixed - + '}'; - } - } - - /** POJO with generic collection. */ - public static class PojoWithCollectionGeneric { - public List pojos; - public IntValue key; - public java.sql.Date sqlDate; - public BigInteger bigInt; - public BigDecimal bigDecimalKeepItNull; - public BigInt scalaBigInt; - public List mixed; - private PojoWithDateAndEnum makeMeGeneric; - - @Override - public String toString() { - return "PojoWithCollection{" - + "pojos.size()=" - + pojos.size() - + ", key=" - + key - + ", sqlDate=" - + sqlDate - + ", bigInt=" - + bigInt - + ", bigDecimalKeepItNull=" - + bigDecimalKeepItNull - + ", scalaBigInt=" - + scalaBigInt - + ", mixed=" - + mixed - + '}'; - } - } - - public static DataSet getPojoWithCollection(ExecutionEnvironment env) { - List data = new ArrayList<>(); - - List pojosList1 = new ArrayList<>(); - pojosList1.add(new Pojo1("a", "aa")); - pojosList1.add(new Pojo1("b", "bb")); - - List pojosList2 = new ArrayList<>(); - pojosList2.add(new Pojo1("a2", "aa2")); - pojosList2.add(new Pojo1("b2", "bb2")); - - PojoWithCollection pwc1 = new PojoWithCollection(); - pwc1.pojos = pojosList1; - pwc1.key = new IntValue(0); - pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN); - pwc1.scalaBigInt = BigInt.int2bigInt(10); - pwc1.bigDecimalKeepItNull = null; - - // use calendar to make it stable across time zones - GregorianCalendar gcl1 = new GregorianCalendar(2033, 04, 18); - pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis()); - pwc1.mixed = new ArrayList(); - Map map = new HashMap<>(); - map.put(new StringValue("someKey"), new IntValue(1)); - pwc1.mixed.add(map); - pwc1.mixed.add(new File("/this/is/wrong")); - pwc1.mixed.add("uhlala"); - - PojoWithCollection pwc2 = new PojoWithCollection(); - pwc2.pojos = pojosList2; - pwc2.key = new IntValue(0); - pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN); - pwc2.scalaBigInt = BigInt.int2bigInt(31104000); - pwc2.bigDecimalKeepItNull = null; - - GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3); - pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976 - - data.add(pwc1); - data.add(pwc2); - - return env.fromCollection(data); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java deleted file mode 100644 index 4c9922d7c4a64..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java +++ /dev/null @@ -1,296 +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.optimizer.examples; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.util.FieldList; -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.api.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.optimizer.util.OperatorResolver; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collection; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -/** Validate the compilation and result of a single iteration of KMeans. */ -@SuppressWarnings("serial") -public class KMeansSingleStepTest extends CompilerTestBase { - - private static final String DATAPOINTS = "Data Points"; - private static final String CENTERS = "Centers"; - - private static final String MAPPER_NAME = "Find Nearest Centers"; - private static final String REDUCER_NAME = "Recompute Center Positions"; - - private static final String SINK = "New Center Positions"; - - private final FieldList set0 = new FieldList(0); - - @Test - public void testCompileKMeansSingleStepWithStats() throws Exception { - - Plan p = getKMeansPlan(); - p.setExecutionConfig(new ExecutionConfig()); - // set the statistics - OperatorResolver cr = getContractResolver(p); - GenericDataSourceBase pointsSource = cr.getNode(DATAPOINTS); - GenericDataSourceBase centersSource = cr.getNode(CENTERS); - setSourceStatistics(pointsSource, 100L * 1024 * 1024 * 1024, 32f); - setSourceStatistics(centersSource, 1024 * 1024, 32f); - - OptimizedPlan plan = compileWithStats(p); - checkPlan(plan); - } - - @Test - public void testCompileKMeansSingleStepWithOutStats() throws Exception { - Plan p = getKMeansPlan(); - p.setExecutionConfig(new ExecutionConfig()); - OptimizedPlan plan = compileNoStats(p); - checkPlan(plan); - } - - private void checkPlan(OptimizedPlan plan) { - - OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); - - final SinkPlanNode sink = or.getNode(SINK); - final SingleInputPlanNode reducer = or.getNode(REDUCER_NAME); - final SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor(); - final SingleInputPlanNode mapper = or.getNode(MAPPER_NAME); - - // check the mapper - assertEquals(1, mapper.getBroadcastInputs().size()); - assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - assertEquals( - ShipStrategyType.BROADCAST, mapper.getBroadcastInputs().get(0).getShipStrategy()); - - assertEquals(LocalStrategy.NONE, mapper.getInput().getLocalStrategy()); - assertEquals(LocalStrategy.NONE, mapper.getBroadcastInputs().get(0).getLocalStrategy()); - - assertEquals(DriverStrategy.MAP, mapper.getDriverStrategy()); - - assertNull(mapper.getInput().getLocalStrategyKeys()); - assertNull(mapper.getInput().getLocalStrategySortOrder()); - assertNull(mapper.getBroadcastInputs().get(0).getLocalStrategyKeys()); - assertNull(mapper.getBroadcastInputs().get(0).getLocalStrategySortOrder()); - - // check the combiner - Assert.assertNotNull(combiner); - assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy()); - assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); - assertNull(combiner.getInput().getLocalStrategyKeys()); - assertNull(combiner.getInput().getLocalStrategySortOrder()); - assertEquals(set0, combiner.getKeys(0)); - assertEquals(set0, combiner.getKeys(1)); - - // check the reducer - assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); - assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy()); - assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reducer.getDriverStrategy()); - assertEquals(set0, reducer.getKeys(0)); - assertEquals(set0, reducer.getInput().getLocalStrategyKeys()); - assertTrue( - Arrays.equals( - reducer.getInput().getLocalStrategySortOrder(), reducer.getSortOrders(0))); - - // check the sink - assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy()); - } - - public static Plan getKMeansPlan() throws Exception { - return kmeans(new String[] {IN_FILE, IN_FILE, OUT_FILE, "20"}); - } - - public static Plan kmeans(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet points = - env.readCsvFile(args[0]) - .fieldDelimiter(" ") - .includeFields(true, true) - .types(Double.class, Double.class) - .name(DATAPOINTS) - .map( - new MapFunction, Point>() { - @Override - public Point map(Tuple2 value) - throws Exception { - return new Point(value.f0, value.f1); - } - }); - - DataSet centroids = - env.readCsvFile(args[1]) - .fieldDelimiter(" ") - .includeFields(true, true, true) - .types(Integer.class, Double.class, Double.class) - .name(CENTERS) - .map( - new MapFunction, Centroid>() { - @Override - public Centroid map(Tuple3 value) - throws Exception { - return new Centroid(value.f0, value.f1, value.f2); - } - }); - - DataSet> newCentroids = - points.map(new SelectNearestCenter()) - .name(MAPPER_NAME) - .withBroadcastSet(centroids, "centroids"); - - DataSet> recomputeClusterCenter = - newCentroids - .groupBy(0) - .reduceGroup(new RecomputeClusterCenter()) - .name(REDUCER_NAME); - - recomputeClusterCenter.project(0, 1).writeAsCsv(args[2], "\n", " ").name(SINK); - - return env.createProgramPlan("KMeans Example"); - } - - /** Two-dimensional point. */ - public static class Point extends Tuple2 { - public Point(double x, double y) { - this.f0 = x; - this.f1 = y; - } - - public Point add(Point other) { - f0 += other.f0; - f1 += other.f1; - return this; - } - - public Point div(long val) { - f0 /= val; - f1 /= val; - return this; - } - - public double euclideanDistance(Point other) { - return Math.sqrt((f0 - other.f0) * (f0 - other.f0) + (f1 - other.f1) * (f1 - other.f1)); - } - - public double euclideanDistance(Centroid other) { - return Math.sqrt( - (f0 - other.f1.f0) * (f0 - other.f1.f0) - + (f1 - other.f1.f1) * (f1 - other.f1.f1)); - } - } - - /** Center of a cluster. */ - public static class Centroid extends Tuple2 { - - public Centroid(int id, double x, double y) { - this.f0 = id; - this.f1 = new Point(x, y); - } - - public Centroid(int id, Point p) { - this.f0 = id; - this.f1 = p; - } - } - - /** Determines the closest cluster center for a data point. */ - private static final class SelectNearestCenter - extends RichMapFunction> { - private Collection centroids; - - @Override - public void open(OpenContext openContext) throws Exception { - this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); - } - - @Override - public Tuple3 map(Point p) throws Exception { - double minDistance = Double.MAX_VALUE; - int closestCentroidId = -1; - for (Centroid centroid : centroids) { - double distance = p.euclideanDistance(centroid); - if (distance < minDistance) { - minDistance = distance; - closestCentroidId = centroid.f0; - } - } - return new Tuple3<>(closestCentroidId, p, 1); - } - } - - private static final class RecomputeClusterCenter - implements GroupReduceFunction< - Tuple3, Tuple3>, - GroupCombineFunction< - Tuple3, Tuple3> { - - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - int id = -1; - double x = 0; - double y = 0; - int count = 0; - for (Tuple3 value : values) { - id = value.f0; - x += value.f1.f0; - y += value.f1.f1; - count += value.f2; - } - out.collect(new Tuple3<>(id, new Point(x, y), count)); - } - - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception { - reduce(values, out); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java deleted file mode 100644 index ccd0ab7a81ba6..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java +++ /dev/null @@ -1,518 +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.optimizer.examples; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.operators.DualInputOperator; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.SingleInputOperator; -import org.apache.flink.api.common.operators.util.FieldList; -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.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.optimizer.util.OperatorResolver; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; - -/** Tests TPCH Q3 (simplified) under various input conditions. */ -@SuppressWarnings("serial") -public class RelationalQueryCompilerTest extends CompilerTestBase { - - private static final String ORDERS = "Orders"; - private static final String LINEITEM = "LineItems"; - private static final String MAPPER_NAME = "FilterO"; - private static final String JOIN_NAME = "JoinLiO"; - private static final String REDUCE_NAME = "AggLiO"; - private static final String SINK = "Output"; - - private final FieldList set0 = new FieldList(0); - private final FieldList set01 = new FieldList(0, 1); - private final ExecutionConfig defaultExecutionConfig = new ExecutionConfig(); - - // ------------------------------------------------------------------------ - - /** - * Verifies that a robust repartitioning plan with a hash join is created in the absence of - * statistics. - */ - @Test - public void testQueryNoStatistics() { - try { - Plan p = getTPCH3Plan(); - p.setExecutionConfig(defaultExecutionConfig); - // compile - final OptimizedPlan plan = compileNoStats(p); - - final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); - - // get the nodes from the final plan - final SinkPlanNode sink = or.getNode(SINK); - final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME); - final SingleInputPlanNode combiner = - reducer.getPredecessor() instanceof SingleInputPlanNode - ? (SingleInputPlanNode) reducer.getPredecessor() - : null; - final DualInputPlanNode join = or.getNode(JOIN_NAME); - final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME); - - // verify the optimizer choices - checkStandardStrategies(filteringMapper, join, combiner, reducer, sink); - Assert.assertTrue(checkRepartitionShipStrategies(join, reducer, combiner)); - Assert.assertTrue( - checkHashJoinStrategies(join, reducer, true) - || checkHashJoinStrategies(join, reducer, false)); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - /** - * Checks if any valid plan is produced. Hash joins are expected to build the orders side, as - * the statistics indicate this to be the smaller one. - */ - @Test - public void testQueryAnyValidPlan() throws Exception { - testQueryGeneric( - 1024 * 1024 * 1024L, - 8 * 1024 * 1024 * 1024L, - 0.05f, - 0.05f, - true, - true, - true, - false, - true); - } - - /** Verifies that the plan compiles in the presence of empty size=0 estimates. */ - @Test - public void testQueryWithSizeZeroInputs() throws Exception { - testQueryGeneric(0, 0, 0.1f, 0.5f, true, true, true, false, true); - } - - /** Statistics that push towards a broadcast join. */ - @Test - public void testQueryWithStatsForBroadcastHash() throws Exception { - testQueryGeneric( - 1024L * 1024 * 1024 * 1024, - 1024L * 1024 * 1024 * 1024, - 0.01f, - 0.05f, - true, - false, - true, - false, - false); - } - - /** Statistics that push towards a broadcast join. */ - @Test - public void testQueryWithStatsForRepartitionAny() throws Exception { - testQueryGeneric( - 100L * 1024 * 1024 * 1024 * 1024, - 100L * 1024 * 1024 * 1024 * 1024, - 0.1f, - 0.5f, - false, - true, - true, - true, - true); - } - - /** - * Statistics that push towards a repartition merge join. If the join blows the data volume up - * significantly, re-exploiting the sorted order is cheaper. - */ - @Test - public void testQueryWithStatsForRepartitionMerge() throws Exception { - Plan p = getTPCH3Plan(); - p.setExecutionConfig(defaultExecutionConfig); - // set compiler hints - OperatorResolver cr = getContractResolver(p); - DualInputOperator match = cr.getNode(JOIN_NAME); - match.getCompilerHints().setFilterFactor(100f); - - testQueryGeneric( - 100L * 1024 * 1024 * 1024 * 1024, - 100L * 1024 * 1024 * 1024 * 1024, - 0.01f, - 100f, - false, - true, - false, - false, - true); - } - - // ------------------------------------------------------------------------ - private void testQueryGeneric( - long orderSize, - long lineItemSize, - float ordersFilterFactor, - float joinFilterFactor, - boolean broadcastOkay, - boolean partitionedOkay, - boolean hashJoinFirstOkay, - boolean hashJoinSecondOkay, - boolean mergeJoinOkay) - throws Exception { - Plan p = getTPCH3Plan(); - p.setExecutionConfig(defaultExecutionConfig); - testQueryGeneric( - p, - orderSize, - lineItemSize, - ordersFilterFactor, - joinFilterFactor, - broadcastOkay, - partitionedOkay, - hashJoinFirstOkay, - hashJoinSecondOkay, - mergeJoinOkay); - } - - private void testQueryGeneric( - Plan p, - long orderSize, - long lineitemSize, - float orderSelectivity, - float joinSelectivity, - boolean broadcastOkay, - boolean partitionedOkay, - boolean hashJoinFirstOkay, - boolean hashJoinSecondOkay, - boolean mergeJoinOkay) { - try { - // set statistics - OperatorResolver cr = getContractResolver(p); - GenericDataSourceBase ordersSource = cr.getNode(ORDERS); - GenericDataSourceBase lineItemSource = cr.getNode(LINEITEM); - SingleInputOperator mapper = cr.getNode(MAPPER_NAME); - DualInputOperator joiner = cr.getNode(JOIN_NAME); - setSourceStatistics(ordersSource, orderSize, 100f); - setSourceStatistics(lineItemSource, lineitemSize, 140f); - mapper.getCompilerHints().setAvgOutputRecordSize(16f); - mapper.getCompilerHints().setFilterFactor(orderSelectivity); - joiner.getCompilerHints().setFilterFactor(joinSelectivity); - - // compile - final OptimizedPlan plan = compileWithStats(p); - final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan); - - // get the nodes from the final plan - final SinkPlanNode sink = or.getNode(SINK); - final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME); - final SingleInputPlanNode combiner = - reducer.getPredecessor() instanceof SingleInputPlanNode - ? (SingleInputPlanNode) reducer.getPredecessor() - : null; - final DualInputPlanNode join = or.getNode(JOIN_NAME); - final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME); - - checkStandardStrategies(filteringMapper, join, combiner, reducer, sink); - - // check the possible variants and that the variant ia allowed in this specific setting - if (checkBroadcastShipStrategies(join, reducer, combiner)) { - Assert.assertTrue("Broadcast join incorrectly chosen.", broadcastOkay); - - if (checkHashJoinStrategies(join, reducer, true)) { - Assert.assertTrue( - "Hash join (build orders) incorrectly chosen", hashJoinFirstOkay); - } else if (checkHashJoinStrategies(join, reducer, false)) { - Assert.assertTrue( - "Hash join (build lineitem) incorrectly chosen", hashJoinSecondOkay); - } else if (checkBroadcastMergeJoin(join, reducer)) { - Assert.assertTrue("Merge join incorrectly chosen", mergeJoinOkay); - } else { - Assert.fail("Plan has no correct hash join or merge join strategies."); - } - } else if (checkRepartitionShipStrategies(join, reducer, combiner)) { - Assert.assertTrue("Partitioned join incorrectly chosen.", partitionedOkay); - - if (checkHashJoinStrategies(join, reducer, true)) { - Assert.assertTrue( - "Hash join (build orders) incorrectly chosen", hashJoinFirstOkay); - } else if (checkHashJoinStrategies(join, reducer, false)) { - Assert.assertTrue( - "Hash join (build lineitem) incorrectly chosen", hashJoinSecondOkay); - } else if (checkRepartitionMergeJoin(join, reducer)) { - Assert.assertTrue("Merge join incorrectly chosen", mergeJoinOkay); - } else { - Assert.fail("Plan has no correct hash join or merge join strategies."); - } - } else { - Assert.fail("Plan has neither correct BC join or partitioned join configuration."); - } - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - // ------------------------------------------------------------------------ - // Checks for special conditions - // ------------------------------------------------------------------------ - - private void checkStandardStrategies( - SingleInputPlanNode map, - DualInputPlanNode join, - SingleInputPlanNode combiner, - SingleInputPlanNode reducer, - SinkPlanNode sink) { - // check ship strategies that are always fix - Assert.assertEquals(ShipStrategyType.FORWARD, map.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - // check the driver strategies that are always fix - Assert.assertEquals(DriverStrategy.FLAT_MAP, map.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reducer.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy()); - if (combiner != null) { - Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); - Assert.assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy()); - } - } - - private boolean checkBroadcastShipStrategies( - DualInputPlanNode join, SingleInputPlanNode reducer, SingleInputPlanNode combiner) { - if (ShipStrategyType.BROADCAST == join.getInput1().getShipStrategy() - && ShipStrategyType.FORWARD == join.getInput2().getShipStrategy() - && ShipStrategyType.PARTITION_HASH == reducer.getInput().getShipStrategy()) { - - // check combiner - Assert.assertNotNull("Plan should have a combiner", combiner); - Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - return true; - } else { - return false; - } - } - - private boolean checkRepartitionShipStrategies( - DualInputPlanNode join, SingleInputPlanNode reducer, SingleInputPlanNode combiner) { - if (ShipStrategyType.PARTITION_HASH == join.getInput1().getShipStrategy() - && ShipStrategyType.PARTITION_HASH == join.getInput2().getShipStrategy() - && ShipStrategyType.FORWARD == reducer.getInput().getShipStrategy()) { - - // check combiner - Assert.assertNull("Plan should not have a combiner", combiner); - return true; - } else { - return false; - } - } - - private boolean checkHashJoinStrategies( - DualInputPlanNode join, SingleInputPlanNode reducer, boolean buildFirst) { - if ((buildFirst && DriverStrategy.HYBRIDHASH_BUILD_FIRST == join.getDriverStrategy()) - || (!buildFirst - && DriverStrategy.HYBRIDHASH_BUILD_SECOND == join.getDriverStrategy())) { - - // driver keys - Assert.assertEquals(set0, join.getKeysForInput1()); - Assert.assertEquals(set0, join.getKeysForInput2()); - - // local strategies - Assert.assertEquals(LocalStrategy.NONE, join.getInput1().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.NONE, join.getInput2().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy()); - - // local strategy keys - Assert.assertEquals(set01, reducer.getInput().getLocalStrategyKeys()); - Assert.assertEquals(set01, reducer.getKeys(0)); - Assert.assertTrue( - Arrays.equals( - reducer.getInput().getLocalStrategySortOrder(), - reducer.getSortOrders(0))); - return true; - } else { - return false; - } - } - - private boolean checkBroadcastMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) { - if (DriverStrategy.INNER_MERGE == join.getDriverStrategy()) { - // driver keys - Assert.assertEquals(set0, join.getKeysForInput1()); - Assert.assertEquals(set0, join.getKeysForInput2()); - - // local strategies - Assert.assertEquals(LocalStrategy.SORT, join.getInput1().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.SORT, join.getInput2().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy()); - - // local strategy keys - Assert.assertEquals(set0, join.getInput1().getLocalStrategyKeys()); - Assert.assertEquals(set0, join.getInput2().getLocalStrategyKeys()); - Assert.assertTrue( - Arrays.equals( - join.getInput1().getLocalStrategySortOrder(), - join.getInput2().getLocalStrategySortOrder())); - Assert.assertEquals(set01, reducer.getInput().getLocalStrategyKeys()); - Assert.assertEquals(set01, reducer.getKeys(0)); - Assert.assertTrue( - Arrays.equals( - reducer.getInput().getLocalStrategySortOrder(), - reducer.getSortOrders(0))); - return true; - } else { - return false; - } - } - - private boolean checkRepartitionMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) { - if (DriverStrategy.INNER_MERGE == join.getDriverStrategy()) { - // driver keys - Assert.assertEquals(set0, join.getKeysForInput1()); - Assert.assertEquals(set0, join.getKeysForInput2()); - - // local strategies - Assert.assertEquals(LocalStrategy.SORT, join.getInput1().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.SORT, join.getInput2().getLocalStrategy()); - Assert.assertEquals(LocalStrategy.NONE, reducer.getInput().getLocalStrategy()); - - // local strategy keys - Assert.assertEquals(set01, join.getInput1().getLocalStrategyKeys()); - Assert.assertEquals(set0, join.getInput2().getLocalStrategyKeys()); - Assert.assertTrue( - join.getInput1().getLocalStrategySortOrder()[0] - == join.getInput2().getLocalStrategySortOrder()[0]); - Assert.assertEquals(set01, reducer.getKeys(0)); - Assert.assertTrue( - Arrays.equals( - join.getInput1().getLocalStrategySortOrder(), - reducer.getSortOrders(0))); - return true; - } else { - return false; - } - } - - public static Plan getTPCH3Plan() throws Exception { - return tpch3(new String[] {DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE}); - } - - public static Plan tpch3(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(Integer.parseInt(args[0])); - - // order id, order status, order data, order prio, ship prio - DataSet> orders = - env.readCsvFile(args[1]) - .fieldDelimiter("|") - .lineDelimiter("\n") - .includeFields("101011001") - .types(Long.class, String.class, String.class, String.class, Integer.class) - .name(ORDERS); - - // order id, extended price - DataSet> lineItems = - env.readCsvFile(args[2]) - .fieldDelimiter("|") - .lineDelimiter("\n") - .includeFields("100001") - .types(Long.class, Double.class) - .name(LINEITEM); - - DataSet> filterO = orders.flatMap(new FilterO()).name(MAPPER_NAME); - - DataSet> joinLiO = - filterO.join(lineItems).where(0).equalTo(0).with(new JoinLiO()).name(JOIN_NAME); - - DataSet> aggLiO = - joinLiO.groupBy(0, 1).reduceGroup(new AggLiO()).name(REDUCE_NAME); - - aggLiO.writeAsCsv(args[3], "\n", "|").name(SINK); - - return env.createProgramPlan(); - } - - @ForwardedFields("f0; f4->f1") - private static class FilterO - implements FlatMapFunction< - Tuple5, Tuple2> { - @Override - public void flatMap( - Tuple5 value, - Collector> out) - throws Exception { - // not going to be executed - } - } - - @ForwardedFieldsFirst("f0; f1") - private static class JoinLiO - implements FlatJoinFunction< - Tuple2, Tuple2, Tuple3> { - @Override - public void join( - Tuple2 first, - Tuple2 second, - Collector> out) - throws Exception { - // not going to be executed - } - } - - @ForwardedFields("f0; f1") - private static class AggLiO - implements GroupReduceFunction< - Tuple3, Tuple3>, - GroupCombineFunction< - Tuple3, Tuple3> { - @Override - public void reduce( - Iterable> values, - Collector> out) - throws Exception { - // not going to be executed - } - - @Override - public void combine( - Iterable> values, - Collector> out) - throws Exception { - // not going to be executed - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java deleted file mode 100644 index ac7d7da4c2fac..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java +++ /dev/null @@ -1,121 +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.optimizer.examples; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.operators.GenericDataSourceBase; -import org.apache.flink.api.common.operators.util.FieldList; -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.optimizer.plan.Channel; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SingleInputPlanNode; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; - -/** Validate program compilation. */ -public class WordCountCompilerTest extends CompilerTestBase { - - private static final long serialVersionUID = 8988304231385358228L; - - /** This method tests the simple word count. */ - @Test - public void testWordCount() { - checkWordCount(true); - checkWordCount(false); - } - - private void checkWordCount(boolean estimates) { - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(DEFAULT_PARALLELISM); - - // get input data - DataSet lines = env.readTextFile(IN_FILE).name("Input Lines"); - - lines - // dummy map - .map( - new MapFunction>() { - private static final long serialVersionUID = -3952739820618875030L; - - @Override - public Tuple2 map(String v) throws Exception { - return new Tuple2<>(v, 1); - } - }) - .name("Tokenize Lines") - // count - .groupBy(0) - .sum(1) - .name("Count Words") - // discard - .output(new DiscardingOutputFormat>()) - .name("Word Counts"); - - // get the plan and compile it - Plan p = env.createProgramPlan(); - p.setExecutionConfig(new ExecutionConfig()); - - OptimizedPlan plan; - if (estimates) { - GenericDataSourceBase source = getContractResolver(p).getNode("Input Lines"); - setSourceStatistics(source, 1024 * 1024 * 1024 * 1024L, 24f); - plan = compileWithStats(p); - } else { - plan = compileNoStats(p); - } - - // get the optimizer plan nodes - OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(plan); - SinkPlanNode sink = resolver.getNode("Word Counts"); - SingleInputPlanNode reducer = resolver.getNode("Count Words"); - SingleInputPlanNode mapper = resolver.getNode("Tokenize Lines"); - - // verify the strategies - Assert.assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy()); - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - - Channel c = reducer.getInput(); - Assert.assertEquals(LocalStrategy.COMBININGSORT, c.getLocalStrategy()); - FieldList l = new FieldList(0); - Assert.assertEquals(l, c.getShipStrategyKeys()); - Assert.assertEquals(l, c.getLocalStrategyKeys()); - Assert.assertTrue(Arrays.equals(c.getLocalStrategySortOrder(), reducer.getSortOrders(0))); - - // check the combiner - SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor(); - Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy()); - Assert.assertEquals(l, combiner.getKeys(0)); - Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy()); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java deleted file mode 100644 index 69afcd88b603e..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java +++ /dev/null @@ -1,236 +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.optimizer.iterations; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.CoGroupFunction; -import org.apache.flink.api.common.functions.FlatJoinFunction; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.operators.util.FieldList; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; -import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.optimizer.dag.TempMode; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.plan.SourcePlanNode; -import org.apache.flink.optimizer.plan.WorksetIterationPlanNode; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; -import org.apache.flink.util.Collector; - -import org.junit.Assert; -import org.junit.Test; - -/** */ -@SuppressWarnings("serial") -public class ConnectedComponentsCoGroupTest extends CompilerTestBase { - - private static final String VERTEX_SOURCE = "Vertices"; - - private static final String ITERATION_NAME = "Connected Components Iteration"; - - private static final String EDGES_SOURCE = "Edges"; - private static final String JOIN_NEIGHBORS_MATCH = "Join Candidate Id With Neighbor"; - private static final String MIN_ID_AND_UPDATE = "Min Id and Update"; - - private static final String SINK = "Result"; - - private static final boolean PRINT_PLAN = false; - - private final FieldList set0 = new FieldList(0); - - @Test - public void testWorksetConnectedComponents() throws Exception { - Plan plan = getConnectedComponentsCoGroupPlan(); - plan.setExecutionConfig(new ExecutionConfig()); - OptimizedPlan optPlan = compileNoStats(plan); - OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan); - - if (PRINT_PLAN) { - PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); - String json = dumper.getOptimizerPlanAsJSON(optPlan); - System.out.println(json); - } - - SourcePlanNode vertexSource = or.getNode(VERTEX_SOURCE); - SourcePlanNode edgesSource = or.getNode(EDGES_SOURCE); - SinkPlanNode sink = or.getNode(SINK); - WorksetIterationPlanNode iter = or.getNode(ITERATION_NAME); - - DualInputPlanNode neighborsJoin = or.getNode(JOIN_NEIGHBORS_MATCH); - DualInputPlanNode cogroup = or.getNode(MIN_ID_AND_UPDATE); - - // -------------------------------------------------------------------- - // Plan validation: - // - // We expect the plan to go with a sort-merge join, because the CoGroup - // sorts and the join in the successive iteration can re-exploit the sorting. - // -------------------------------------------------------------------- - - // test all drivers - Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, vertexSource.getDriverStrategy()); - Assert.assertEquals(DriverStrategy.NONE, edgesSource.getDriverStrategy()); - - Assert.assertEquals(DriverStrategy.INNER_MERGE, neighborsJoin.getDriverStrategy()); - Assert.assertEquals(set0, neighborsJoin.getKeysForInput1()); - Assert.assertEquals(set0, neighborsJoin.getKeysForInput2()); - - Assert.assertEquals(DriverStrategy.CO_GROUP, cogroup.getDriverStrategy()); - Assert.assertEquals(set0, cogroup.getKeysForInput1()); - Assert.assertEquals(set0, cogroup.getKeysForInput2()); - - // test all the shipping strategies - Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, - iter.getInitialSolutionSetInput().getShipStrategy()); - Assert.assertEquals(set0, iter.getInitialSolutionSetInput().getShipStrategyKeys()); - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, iter.getInitialWorksetInput().getShipStrategy()); - Assert.assertEquals(set0, iter.getInitialWorksetInput().getShipStrategyKeys()); - - Assert.assertEquals( - ShipStrategyType.FORWARD, neighborsJoin.getInput1().getShipStrategy()); // workset - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, - neighborsJoin.getInput2().getShipStrategy()); // edges - Assert.assertEquals(set0, neighborsJoin.getInput2().getShipStrategyKeys()); - Assert.assertTrue(neighborsJoin.getInput2().getTempMode().isCached()); - - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, cogroup.getInput1().getShipStrategy()); // min id - Assert.assertEquals( - ShipStrategyType.FORWARD, cogroup.getInput2().getShipStrategy()); // solution set - - // test all the local strategies - Assert.assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy()); - Assert.assertEquals( - LocalStrategy.NONE, iter.getInitialSolutionSetInput().getLocalStrategy()); - - // the sort for the neighbor join in the first iteration is pushed out of the loop - Assert.assertEquals(LocalStrategy.SORT, iter.getInitialWorksetInput().getLocalStrategy()); - Assert.assertEquals( - LocalStrategy.NONE, neighborsJoin.getInput1().getLocalStrategy()); // workset - Assert.assertEquals( - LocalStrategy.SORT, neighborsJoin.getInput2().getLocalStrategy()); // edges - - Assert.assertEquals(LocalStrategy.SORT, cogroup.getInput1().getLocalStrategy()); - Assert.assertEquals( - LocalStrategy.NONE, cogroup.getInput2().getLocalStrategy()); // solution set - - // check the caches - Assert.assertTrue(TempMode.CACHED == neighborsJoin.getInput2().getTempMode()); - - JobGraphGenerator jgg = new JobGraphGenerator(); - jgg.compileJobGraph(optPlan); - } - - public static Plan getConnectedComponentsCoGroupPlan() throws Exception { - return connectedComponentsWithCoGroup( - new String[] {DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE, "100"}); - } - - public static Plan connectedComponentsWithCoGroup(String[] args) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(Integer.parseInt(args[0])); - - DataSet> initialVertices = - env.readCsvFile(args[1]).types(Long.class).name(VERTEX_SOURCE); - - DataSet> edges = - env.readCsvFile(args[2]).types(Long.class, Long.class).name(EDGES_SOURCE); - - DataSet> verticesWithId = - initialVertices.flatMap(new DummyMapFunction()); - - DeltaIteration, Tuple2> iteration = - verticesWithId - .iterateDelta(verticesWithId, Integer.parseInt(args[4]), 0) - .name(ITERATION_NAME); - - DataSet> joinWithNeighbors = - iteration - .getWorkset() - .join(edges) - .where(0) - .equalTo(0) - .with(new DummyJoinFunction()) - .name(JOIN_NEIGHBORS_MATCH); - - DataSet> minAndUpdate = - joinWithNeighbors - .coGroup(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new DummyCoGroupFunction()) - .name(MIN_ID_AND_UPDATE); - - iteration.closeWith(minAndUpdate, minAndUpdate).writeAsCsv(args[3]).name(SINK); - - return env.createProgramPlan(); - } - - private static class DummyMapFunction - implements FlatMapFunction, Tuple2> { - @Override - public void flatMap(Tuple1 value, Collector> out) - throws Exception { - // won't be executed - } - } - - private static class DummyJoinFunction - implements FlatJoinFunction< - Tuple2, Tuple2, Tuple2> { - @Override - public void join( - Tuple2 first, - Tuple2 second, - Collector> out) - throws Exception { - // won't be executed - } - } - - @ForwardedFieldsFirst("f0->f0") - @ForwardedFieldsSecond("f0->f0") - private static class DummyCoGroupFunction - implements CoGroupFunction, Tuple2, Tuple2> { - @Override - public void coGroup( - Iterable> first, - Iterable> second, - Collector> out) - throws Exception { - // won't be executed - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java deleted file mode 100644 index cf5d0ffdcea23..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java +++ /dev/null @@ -1,163 +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.optimizer.iterations; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.functions.RichJoinFunction; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.operators.DeltaIteration; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.optimizer.plan.DualInputPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SolutionSetPlanNode; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.DriverStrategy; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.util.Collector; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -/** Test multiple joins with the solution set. */ -@SuppressWarnings("serial") -public class MultipleJoinsWithSolutionSetCompilerTest extends CompilerTestBase { - - private static final String JOIN_1 = "join1"; - private static final String JOIN_2 = "join2"; - - @Test - public void testMultiSolutionSetJoinPlan() { - try { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - @SuppressWarnings("unchecked") - DataSet> inputData = - env.fromElements(new Tuple2(1L, 1.0)); - DataSet> result = constructPlan(inputData, 10); - - // add two sinks, to test the case of branching after an iteration - result.output(new DiscardingOutputFormat>()); - result.output(new DiscardingOutputFormat>()); - - Plan p = env.createProgramPlan(); - - OptimizedPlan optPlan = compileNoStats(p); - - OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan); - - DualInputPlanNode join1 = or.getNode(JOIN_1); - DualInputPlanNode join2 = or.getNode(JOIN_2); - - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, join1.getDriverStrategy()); - assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, join2.getDriverStrategy()); - - assertEquals(ShipStrategyType.PARTITION_HASH, join1.getInput2().getShipStrategy()); - assertEquals(ShipStrategyType.PARTITION_HASH, join2.getInput1().getShipStrategy()); - - assertEquals(SolutionSetPlanNode.class, join1.getInput1().getSource().getClass()); - assertEquals(SolutionSetPlanNode.class, join2.getInput2().getSource().getClass()); - - new JobGraphGenerator().compileJobGraph(optPlan); - } catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Test erroneous: " + e.getMessage()); - } - } - - public static DataSet> constructPlan( - DataSet> initialData, int numIterations) { - - DeltaIteration, Tuple2> iteration = - initialData.iterateDelta(initialData, numIterations, 0); - - DataSet> delta = - iteration - .getSolutionSet() - .join(iteration.getWorkset().flatMap(new Duplicator())) - .where(0) - .equalTo(0) - .with(new SummingJoin()) - .name(JOIN_1) - .groupBy(0) - .aggregate(Aggregations.MIN, 1) - .map(new Expander()) - .join(iteration.getSolutionSet()) - .where(0) - .equalTo(0) - .with(new SummingJoinProject()) - .name(JOIN_2); - - DataSet> changes = delta.groupBy(0).aggregate(Aggregations.SUM, 1); - - DataSet> result = iteration.closeWith(delta, changes); - - return result; - } - - private static final class SummingJoin - extends RichJoinFunction< - Tuple2, Tuple2, Tuple2> { - - @Override - public Tuple2 join(Tuple2 first, Tuple2 second) { - return new Tuple2(first.f0, first.f1 + second.f1); - } - } - - private static final class SummingJoinProject - extends RichJoinFunction< - Tuple3, Tuple2, Tuple2> { - - @Override - public Tuple2 join( - Tuple3 first, Tuple2 second) { - return new Tuple2(first.f0, first.f1 + first.f2 + second.f1); - } - } - - private static final class Duplicator - extends RichFlatMapFunction, Tuple2> { - - @Override - public void flatMap(Tuple2 value, Collector> out) { - out.collect(value); - out.collect(value); - } - } - - private static final class Expander - extends RichMapFunction, Tuple3> { - - @Override - public Tuple3 map(Tuple2 value) { - return new Tuple3(value.f0, value.f1, value.f1 * 2); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java deleted file mode 100644 index a5ec251b742a3..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java +++ /dev/null @@ -1,124 +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.optimizer.iterations; - -import org.apache.flink.api.common.Plan; -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.operators.IterativeDataSet; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.examples.java.graph.PageRank.BuildOutgoingEdgeList; -import org.apache.flink.examples.java.graph.PageRank.Dampener; -import org.apache.flink.examples.java.graph.PageRank.EpsilonFilter; -import org.apache.flink.examples.java.graph.PageRank.JoinVertexWithEdgesMatch; -import org.apache.flink.examples.java.graph.PageRank.RankAssigner; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.BulkIterationPlanNode; -import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.SinkPlanNode; -import org.apache.flink.optimizer.util.CompilerTestBase; -import org.apache.flink.runtime.operators.shipping.ShipStrategyType; -import org.apache.flink.runtime.operators.util.LocalStrategy; - -import org.junit.Assert; -import org.junit.Test; - -import static org.apache.flink.api.java.aggregation.Aggregations.SUM; -import static org.junit.Assert.fail; - -/** Test compilation of PageRank implementation. */ -public class PageRankCompilerTest extends CompilerTestBase { - - @Test - public void testPageRank() { - try { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - // get input data - DataSet pagesInput = env.fromElements(1L); - @SuppressWarnings("unchecked") - DataSet> linksInput = - env.fromElements(new Tuple2(1L, 2L)); - - // assign initial rank to pages - DataSet> pagesWithRanks = - pagesInput.map(new RankAssigner((1.0d / 10))); - - // build adjacency list from link input - DataSet> adjacencyListInput = - linksInput.groupBy(0).reduceGroup(new BuildOutgoingEdgeList()); - - // set iterative data set - IterativeDataSet> iteration = pagesWithRanks.iterate(10); - - Configuration cfg = new Configuration(); - cfg.setString( - Optimizer.HINT_LOCAL_STRATEGY, Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND); - - DataSet> newRanks = - iteration - // join pages with outgoing edges and distribute rank - .join(adjacencyListInput) - .where(0) - .equalTo(0) - .withParameters(cfg) - .flatMap(new JoinVertexWithEdgesMatch()) - // collect and sum ranks - .groupBy(0) - .aggregate(SUM, 1) - // apply dampening factor - .map(new Dampener(0.85, 10)); - - DataSet> finalPageRanks = - iteration.closeWith( - newRanks, - newRanks.join(iteration) - .where(0) - .equalTo(0) - // termination condition - .filter(new EpsilonFilter())); - - finalPageRanks.output(new DiscardingOutputFormat>()); - - // get the plan and compile it - Plan p = env.createProgramPlan(); - OptimizedPlan op = compileNoStats(p); - - SinkPlanNode sinkPlanNode = (SinkPlanNode) op.getDataSinks().iterator().next(); - BulkIterationPlanNode iterPlanNode = - (BulkIterationPlanNode) sinkPlanNode.getInput().getSource(); - - // check that the partitioning is pushed out of the first loop - Assert.assertEquals( - ShipStrategyType.PARTITION_HASH, iterPlanNode.getInput().getShipStrategy()); - Assert.assertEquals(LocalStrategy.NONE, iterPlanNode.getInput().getLocalStrategy()); - - BulkPartialSolutionPlanNode partSolPlanNode = iterPlanNode.getPartialSolutionPlanNode(); - Assert.assertEquals( - ShipStrategyType.FORWARD, - partSolPlanNode.getOutgoingChannels().get(0).getShipStrategy()); - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java deleted file mode 100644 index 11f1e40877b15..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.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.test.optimizer.jsonplan; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.dag.Pipeline; -import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.client.program.PackagedProgramUtils; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.examples.java.clustering.KMeans; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.examples.java.graph.PageRank; -import org.apache.flink.examples.java.relational.TPCHQuery3; -import org.apache.flink.examples.java.relational.WebLogAnalysis; -import org.apache.flink.examples.java.wordcount.WordCount; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; - -/** The tests in this class simply invokes the JSON dump code for the optimized plan. */ -public class DumpCompiledPlanTest extends CompilerTestBase { - - @Test - public void dumpWordCount() throws Exception { - verifyOptimizedPlan(WordCount.class, "--input", IN_FILE, "--output", OUT_FILE); - } - - @Test - public void dumpTPCH3() throws Exception { - verifyOptimizedPlan( - TPCHQuery3.class, - "--lineitem", - IN_FILE, - "--customer", - IN_FILE, - "--orders", - OUT_FILE, - "--output", - "123"); - } - - @Test - public void dumpIterativeKMeans() throws Exception { - verifyOptimizedPlan( - KMeans.class, - "--points ", - IN_FILE, - "--centroids ", - IN_FILE, - "--output ", - OUT_FILE, - "--iterations", - "123"); - } - - @Test - public void dumpWebLogAnalysis() throws Exception { - verifyOptimizedPlan( - WebLogAnalysis.class, - "--documents", - IN_FILE, - "--ranks", - IN_FILE, - "--visits", - OUT_FILE, - "--output", - "123"); - } - - @Test - public void dumpBulkIterationKMeans() throws Exception { - verifyOptimizedPlan( - ConnectedComponents.class, - "--vertices", - IN_FILE, - "--edges", - IN_FILE, - "--output", - OUT_FILE, - "--iterations", - "123"); - } - - @Test - public void dumpPageRank() throws Exception { - verifyOptimizedPlan( - PageRank.class, - "--pages", - IN_FILE, - "--links", - IN_FILE, - "--output", - OUT_FILE, - "--numPages", - "10", - "--iterations", - "123"); - } - - private void verifyOptimizedPlan(Class entrypoint, String... args) throws Exception { - final PackagedProgram program = - PackagedProgram.newBuilder() - .setEntryPointClassName(entrypoint.getName()) - .setArguments(args) - .build(); - - final Pipeline pipeline = - PackagedProgramUtils.getPipelineFromProgram(program, new Configuration(), 1, true); - - assertTrue(pipeline instanceof Plan); - - final Plan plan = (Plan) pipeline; - - final OptimizedPlan op = compileNoStats(plan); - final PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); - final String json = dumper.getOptimizerPlanAsJSON(op); - try (JsonParser parser = new JsonFactory().createParser(json)) { - while (parser.nextToken() != null) {} - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java deleted file mode 100644 index 3e7a589f779ae..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java +++ /dev/null @@ -1,383 +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.optimizer.jsonplan; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.ExecutionEnvironmentFactory; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.examples.java.clustering.KMeans; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.examples.java.relational.WebLogAnalysis; -import org.apache.flink.examples.java.wordcount.WordCount; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; - -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.OutputStream; -import java.io.PrintStream; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** Test job graph generation in JSON format. */ -public class JsonJobGraphGenerationTest { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private PrintStream out; - private PrintStream err; - - @Before - public void redirectStreams() { - this.out = System.out; - this.err = System.err; - - OutputStream discards = - new OutputStream() { - @Override - public void write(int b) {} - }; - - System.setOut(new PrintStream(discards)); - System.setErr(new PrintStream(discards)); - } - - @After - public void restoreStreams() { - if (out != null) { - System.setOut(out); - } - if (err != null) { - System.setOut(err); - } - } - - @AfterClass - public static void resetContextEnvironment() { - TestingExecutionEnvironment.unset(); - } - - @Test - public void testWordCountPlan() { - try { - // without arguments - try { - final int parallelism = 1; // some ops have DOP 1 forced - JsonValidator validator = new GenericValidator(parallelism, 3); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - WordCount.main(new String[0]); - } catch (AbortError ignored) { - } - - // with arguments - try { - final int parallelism = 17; - JsonValidator validator = new GenericValidator(parallelism, 3); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - String tmpDir = tempFolder.newFolder().getAbsolutePath(); - WordCount.main( - new String[] { - "--input", tmpDir, - "--output", tmpDir - }); - } catch (AbortError ignored) { - } - } catch (Exception e) { - restoreStreams(); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testWebLogAnalysis() { - try { - // without arguments - try { - final int parallelism = 1; // some ops have DOP 1 forced - JsonValidator validator = new GenericValidator(parallelism, 6); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - WebLogAnalysis.main(new String[0]); - } catch (AbortError ignored) { - } - - // with arguments - try { - final int parallelism = 17; - JsonValidator validator = new GenericValidator(parallelism, 6); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - String tmpDir = tempFolder.newFolder().getAbsolutePath(); - WebLogAnalysis.main( - new String[] { - "--documents", tmpDir, - "--ranks", tmpDir, - "--visits", tmpDir, - "--output", tmpDir - }); - } catch (AbortError ignored) { - } - } catch (Exception e) { - restoreStreams(); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testKMeans() { - try { - // without arguments - try { - final int parallelism = 1; // some ops have DOP 1 forced - JsonValidator validator = new GenericValidator(parallelism, 9); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - KMeans.main(new String[0]); - } catch (AbortError ignored) { - } - - // with arguments - try { - final int parallelism = 42; - JsonValidator validator = new GenericValidator(parallelism, 9); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - String tmpDir = tempFolder.newFolder().getAbsolutePath(); - KMeans.main( - new String[] { - "--points", tmpDir, - "--centroids", tmpDir, - "--output", tmpDir, - "--iterations", "100" - }); - } catch (AbortError ignored) { - } - - } catch (Exception e) { - restoreStreams(); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testConnectedComponents() { - try { - // without arguments - try { - final int parallelism = 1; // some ops have DOP 1 forced - JsonValidator validator = new GenericValidator(parallelism, 9); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - ConnectedComponents.main(); - } catch (AbortError ignored) { - } - - // with arguments - try { - final int parallelism = 23; - JsonValidator validator = new GenericValidator(parallelism, 9); - TestingExecutionEnvironment.setAsNext(validator, parallelism); - - String tmpDir = tempFolder.newFolder().getAbsolutePath(); - ConnectedComponents.main( - "--vertices", tmpDir, - "--edges", tmpDir, - "--output", tmpDir, - "--iterations", "100"); - } catch (AbortError ignored) { - } - - } catch (Exception e) { - restoreStreams(); - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // ------------------------------------------------------------------------ - - private interface JsonValidator { - - void validateJson(String json) throws Exception; - } - - private static class GenericValidator implements JsonValidator { - - private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); - - private final int expectedParallelism; - private final int numNodes; - - GenericValidator(int expectedParallelism, int numNodes) { - this.expectedParallelism = expectedParallelism; - this.numNodes = numNodes; - } - - @Override - public void validateJson(String json) throws Exception { - final Map idToNode = new HashMap<>(); - - // validate the produced JSON - JsonNode rootNode = OBJECT_MAPPER.readTree(json); - - JsonNode idField = rootNode.get("jid"); - JsonNode nameField = rootNode.get("name"); - JsonNode typeField = rootNode.get("type"); - JsonNode arrayField = rootNode.get("nodes"); - - assertNotNull(idField); - assertNotNull(nameField); - assertNotNull(typeField); - assertNotNull(arrayField); - assertTrue(idField.isTextual()); - assertTrue(nameField.isTextual()); - assertTrue(typeField.isTextual()); - assertTrue(arrayField.isArray()); - - ArrayNode array = (ArrayNode) arrayField; - Iterator iter = array.elements(); - while (iter.hasNext()) { - JsonNode vertex = iter.next(); - - JsonNode vertexIdField = vertex.get("id"); - JsonNode parallelismField = vertex.get("parallelism"); - JsonNode contentsFields = vertex.get("description"); - JsonNode operatorField = vertex.get("operator"); - - assertNotNull(vertexIdField); - assertTrue(vertexIdField.isTextual()); - assertNotNull(parallelismField); - assertTrue(parallelismField.isNumber()); - assertNotNull(contentsFields); - assertTrue(contentsFields.isTextual()); - assertNotNull(operatorField); - assertTrue(operatorField.isTextual()); - - if (contentsFields.asText().startsWith("Sync")) { - assertEquals(1, parallelismField.asInt()); - } else { - assertEquals(expectedParallelism, parallelismField.asInt()); - } - - idToNode.put(vertexIdField.asText(), vertex); - } - - assertEquals(numNodes, idToNode.size()); - - // check that all inputs are contained - for (JsonNode node : idToNode.values()) { - JsonNode inputsField = node.get("inputs"); - if (inputsField != null) { - Iterator inputsIter = inputsField.elements(); - while (inputsIter.hasNext()) { - JsonNode inputNode = inputsIter.next(); - JsonNode inputIdField = inputNode.get("id"); - - assertNotNull(inputIdField); - assertTrue(inputIdField.isTextual()); - - String inputIdString = inputIdField.asText(); - assertTrue(idToNode.containsKey(inputIdString)); - } - } - } - } - } - - // ------------------------------------------------------------------------ - - private static class AbortError extends Error { - private static final long serialVersionUID = 152179957828703919L; - } - - // ------------------------------------------------------------------------ - - private static class TestingExecutionEnvironment extends ExecutionEnvironment { - - private final JsonValidator validator; - - private TestingExecutionEnvironment(JsonValidator validator) { - this.validator = validator; - } - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - Plan plan = createProgramPlan(jobName); - - Optimizer pc = new Optimizer(new Configuration()); - OptimizedPlan op = pc.compile(plan); - - JobGraphGenerator jgg = new JobGraphGenerator(); - JobGraph jobGraph = jgg.compileJobGraph(op); - - String jsonPlan = JsonPlanGenerator.generatePlan(jobGraph); - - // first check that the JSON is valid - JsonParser parser = new JsonFactory().createJsonParser(jsonPlan); - while (parser.nextToken() != null) {} - - validator.validateJson(jsonPlan); - - throw new AbortError(); - } - - public static void setAsNext(final JsonValidator validator, final int defaultParallelism) { - initializeContextEnvironment( - new ExecutionEnvironmentFactory() { - @Override - public ExecutionEnvironment createExecutionEnvironment() { - ExecutionEnvironment env = new TestingExecutionEnvironment(validator); - env.setParallelism(defaultParallelism); - return env; - } - }); - } - - public static void unset() { - ExecutionEnvironment.resetContextEnvironment(); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java deleted file mode 100644 index 03d81a61c1cba..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java +++ /dev/null @@ -1,148 +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.optimizer.jsonplan; - -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.dag.Pipeline; -import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.client.program.PackagedProgramUtils; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.examples.java.clustering.KMeans; -import org.apache.flink.examples.java.graph.ConnectedComponents; -import org.apache.flink.examples.java.graph.PageRank; -import org.apache.flink.examples.java.relational.TPCHQuery3; -import org.apache.flink.examples.java.relational.WebLogAnalysis; -import org.apache.flink.examples.java.wordcount.WordCount; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.dag.DataSinkNode; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.optimizer.util.CompilerTestBase; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; - -import org.junit.Test; - -import java.util.List; - -import static org.junit.Assert.assertTrue; - -/** The tests in this class simply invokes the JSON dump code for the original plan. */ -public class PreviewPlanDumpTest extends CompilerTestBase { - - @Test - public void dumpWordCount() throws Exception { - verifyPlanDump(WordCount.class, "--input", IN_FILE, "--output", OUT_FILE); - } - - @Test - public void dumpTPCH3() throws Exception { - verifyPlanDump( - TPCHQuery3.class, - "--lineitem", - IN_FILE, - "--customer", - IN_FILE, - "--orders", - OUT_FILE, - "--output", - "123"); - } - - @Test - public void dumpIterativeKMeans() throws Exception { - verifyPlanDump( - KMeans.class, - "--points ", - IN_FILE, - "--centroids ", - IN_FILE, - "--output ", - OUT_FILE, - "--iterations", - "123"); - } - - @Test - public void dumpWebLogAnalysis() throws Exception { - verifyPlanDump( - WebLogAnalysis.class, - "--documents", - IN_FILE, - "--ranks", - IN_FILE, - "--visits", - OUT_FILE, - "--output", - "123"); - } - - @Test - public void dumpBulkIterationKMeans() throws Exception { - verifyPlanDump( - ConnectedComponents.class, - "--vertices", - IN_FILE, - "--edges", - IN_FILE, - "--output", - OUT_FILE, - "--iterations", - "123"); - } - - @Test - public void dumpPageRank() throws Exception { - verifyPlanDump( - PageRank.class, - "--pages", - IN_FILE, - "--links", - IN_FILE, - "--output", - OUT_FILE, - "--numPages", - "10", - "--iterations", - "123"); - } - - private static void verifyPlanDump(Class entrypoint, String... args) throws Exception { - final PackagedProgram program = - PackagedProgram.newBuilder() - .setEntryPointClassName(entrypoint.getName()) - .setArguments(args) - .build(); - - final Pipeline pipeline = - PackagedProgramUtils.getPipelineFromProgram(program, new Configuration(), 1, true); - - assertTrue(pipeline instanceof Plan); - - final Plan plan = (Plan) pipeline; - - final List sinks = Optimizer.createPreOptimizedPlan(plan); - final PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); - final String json = dumper.getPactPlanAsJSON(sinks); - - try (JsonParser parser = new JsonFactory().createParser(json)) { - while (parser.nextToken() != null) {} - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java deleted file mode 100644 index f372633dfbbbd..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java +++ /dev/null @@ -1,85 +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.planning; - -import org.apache.flink.api.common.functions.RichMapFunction; -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.io.DiscardingOutputFormat; -import org.apache.flink.api.java.tuple.Tuple2; - -import org.junit.Test; - -/** Tests that large programs can be compiled to a Plan in reasonable amount of time. */ -public class LargePlanTest { - - @Test(timeout = 30_000) - public void testPlanningOfLargePlan() throws Exception { - runProgram(10, 20); - } - - private static void runProgram(int depth, int width) throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet input = env.fromElements("a", "b", "c"); - DataSet stats = null; - - for (int i = 0; i < depth; i++) { - stats = analyze(input, stats, width / (i + 1) + 1); - } - - stats.output(new DiscardingOutputFormat<>()); - - env.createProgramPlan("depth " + depth + " width " + width); - } - - private static DataSet analyze( - DataSet input, DataSet stats, int branches) { - for (int i = 0; i < branches; i++) { - final int ii = i; - - if (stats != null) { - input = - input.map( - new RichMapFunction() { - @Override - public String map(String value) { - return value; - } - }) - .withBroadcastSet(stats.map(s -> "(" + s + ").map"), "stats"); - } - - DataSet branch = - input.map(s -> new Tuple2<>(0, s + ii)) - .returns(Types.TUPLE(Types.STRING, Types.INT)) - .groupBy(0) - .minBy(1) - .map(kv -> kv.f1) - .returns(Types.STRING); - if (stats == null) { - stats = branch; - } else { - stats = stats.union(branch); - } - } - return stats.map(s -> "(" + s + ").stats"); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java index 5d9b2a97cb58d..b46910033dbd8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java @@ -20,9 +20,7 @@ import org.apache.flink.api.common.ExecutionMode; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.functions.RichMapPartitionFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -49,7 +47,14 @@ import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptDetailsInfo; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.TestEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow; +import org.apache.flink.streaming.util.RestartStrategyUtils; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Collector; import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.ExceptionUtils; @@ -128,10 +133,9 @@ public class BatchFineGrainedRecoveryITCase extends TestLogger { private static final int EMITTED_RECORD_NUMBER = 1000; private static final int MAP_NUMBER = 3; - private static final String MAP_PARTITION_TEST_PARTITION_MAPPER = - "MapPartition (Test partition mapper "; + private static final String MAP_PARTITION_TEST_PARTITION_MAPPER = "Test partition mapper "; private static final Pattern MAPPER_NUMBER_IN_TASK_NAME_PATTERN = - Pattern.compile("MapPartition \\(Test partition mapper (\\d+)\\)"); + Pattern.compile("Test partition mapper (\\d+)"); /** * Number of job failures for all mappers due to backtracking when the produced partitions get @@ -215,18 +219,21 @@ public void teardown() throws Exception { @Test public void testProgram() throws Exception { - ExecutionEnvironment env = createExecutionEnvironment(); - - DataSet input = env.generateSequence(0, EMITTED_RECORD_NUMBER - 1); + StreamExecutionEnvironment env = createExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + DataStream input = env.fromSequence(0, EMITTED_RECORD_NUMBER - 1); + env.disableOperatorChaining(); for (int trackingIndex = 0; trackingIndex < MAP_NUMBER; trackingIndex++) { input = - input.mapPartition( + input.windowAll(GlobalWindows.createWithEndOfStreamTrigger()) + .apply( new TestPartitionMapper( trackingIndex, createFailureStrategy(trackingIndex))) .name(TASK_NAME_PREFIX + trackingIndex); } - assertThat(input.collect(), is(EXPECTED_JOB_OUTPUT)); + assertThat( + CollectionUtil.iteratorToList(input.executeAndCollect()), is(EXPECTED_JOB_OUTPUT)); failureTracker.verify(getMapperAttempts()); } @@ -262,10 +269,11 @@ private static FailureStrategy createFailureStrategy(int trackingIndex) { return failureStrategy; } - private static ExecutionEnvironment createExecutionEnvironment() { + private static StreamExecutionEnvironment createExecutionEnvironment() { @SuppressWarnings("StaticVariableUsedBeforeInitialization") - ExecutionEnvironment env = new TestEnvironment(miniCluster, 1, true); - + StreamExecutionEnvironment env = new TestStreamEnvironment(miniCluster, 1); + RestartStrategyUtils.configureFixedDelayRestartStrategy( + env, MAX_JOB_RESTART_ATTEMPTS, Duration.ofMillis(10)); env.getConfig() .setExecutionMode( ExecutionMode.BATCH_FORCED); // forces all partitions to be blocking @@ -536,7 +544,8 @@ private void verify(int[] mapAttemptNumbers) { } } - private static class TestPartitionMapper extends RichMapPartitionFunction { + private static class TestPartitionMapper + implements AllWindowFunction { private static final long serialVersionUID = 1L; private final int trackingIndex; @@ -548,7 +557,8 @@ private TestPartitionMapper(int trackingIndex, FailureStrategy failureStrategy) } @Override - public void mapPartition(Iterable values, Collector out) throws Exception { + public void apply(GlobalWindow window, Iterable values, Collector out) + throws Exception { for (Long value : values) { failureStrategy.failOrNot(trackingIndex); out.collect(value + 1); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index cb6355f381200..1ef25bb32b21b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -18,14 +18,11 @@ package org.apache.flink.test.recovery; -import org.apache.flink.api.common.ExecutionMode; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.time.Deadline; -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.HighAvailabilityOptions; import org.apache.flink.configuration.MemorySize; @@ -50,11 +47,12 @@ import org.apache.flink.runtime.testutils.DispatcherProcess; import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperExtension; +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.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorExtension; -import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; -import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; -import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.apache.flink.testutils.junit.extensions.parameterized.NoOpTestExtension; import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.Collector; import org.apache.flink.util.concurrent.FutureUtils; @@ -68,8 +66,6 @@ import java.io.File; import java.time.Duration; -import java.util.Arrays; -import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -94,7 +90,7 @@ *

This follows the same structure as {@link AbstractTaskManagerProcessFailureRecoveryTest}. */ @SuppressWarnings("serial") -@ExtendWith(ParameterizedTestExtension.class) +@ExtendWith(NoOpTestExtension.class) class JobManagerHAProcessFailureRecoveryITCase { private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension(); @@ -117,17 +113,6 @@ class JobManagerHAProcessFailureRecoveryITCase { protected static final int PARALLELISM = 4; - // -------------------------------------------------------------------------------------------- - // Parametrization (run pipelined and batch) - // -------------------------------------------------------------------------------------------- - - @Parameter private ExecutionMode executionMode; - - @Parameters(name = "executionMode={0}") - private static Collection executionMode() { - return Arrays.asList(ExecutionMode.PIPELINED, ExecutionMode.BATCH); - } - /** * Test program with JobManager failure. * @@ -143,8 +128,8 @@ private void testJobManagerFailure( config.set(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkQuorum); config.set(HighAvailabilityOptions.HA_STORAGE_PATH, zookeeperStoragePath.getAbsolutePath()); - ExecutionEnvironment env = - ExecutionEnvironment.createRemoteEnvironment("leader", 1, config); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.createRemoteEnvironment("leader", 1, config); env.setParallelism(PARALLELISM); Configuration configuration = new Configuration(); configuration.set(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay"); @@ -153,11 +138,11 @@ private void testJobManagerFailure( RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofMillis(0)); env.configure(configuration, Thread.currentThread().getContextClassLoader()); - env.getConfig().setExecutionMode(executionMode); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); final long numElements = 100000L; - final DataSet result = - env.generateSequence(1, numElements) + final DataStream result = + env.fromSequence(1, numElements) // make sure every mapper is involved (no one is skipped because of lazy // split assignment) .rebalance() @@ -197,6 +182,9 @@ public Long map(Long value) throws Exception { return value; } }) + .setParallelism(PARALLELISM) + // we map all data to single partition to force 1 parallelism reduce. + .keyBy(x -> 0) .reduce( new ReduceFunction() { @Override @@ -204,6 +192,7 @@ public Long reduce(Long value1, Long value2) { return value1 + value2; } }) + .setParallelism(1) // The check is done in the mapper, because the client can currently not // handle // job manager losses/reconnects. @@ -224,9 +213,10 @@ public void flatMap(Long value, Collector out) coordinateDir, FINISH_MARKER_FILE_PREFIX + taskIndex)); } - }); + }) + .setParallelism(1); - result.output(new DiscardingOutputFormat<>()); + result.sinkTo(new DiscardingSink<>()); env.execute(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index 081cd0cfc45e6..fb109741f9454 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -19,8 +19,6 @@ package org.apache.flink.test.recovery; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -53,6 +51,8 @@ import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.impl.VoidMetricQueryServiceRetriever; import org.apache.flink.runtime.zookeeper.ZooKeeperExtension; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.test.recovery.utils.TaskExecutorProcessEntryPoint; import org.apache.flink.test.util.TestProcessBuilder; import org.apache.flink.test.util.TestProcessBuilder.TestProcess; @@ -176,8 +176,8 @@ void testCancelingOnProcessFailure() throws Throwable { @Override public void run() { try { - ExecutionEnvironment env = - ExecutionEnvironment.createRemoteEnvironment( + StreamExecutionEnvironment env = + StreamExecutionEnvironment.createRemoteEnvironment( "localhost", 1337, config); env.setParallelism(2); Configuration configuration = new Configuration(); @@ -186,7 +186,7 @@ public void run() { configuration, Thread.currentThread().getContextClassLoader()); - env.generateSequence(0, Long.MAX_VALUE) + env.fromSequence(0, Long.MAX_VALUE) .map( new MapFunction() { @@ -200,7 +200,7 @@ public Long map(Long value) throws Exception { return 0L; } }) - .output(new DiscardingOutputFormat<>()); + .sinkTo(new DiscardingSink<>()); env.execute(); } catch (Throwable t) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java index 472fe93f5d9f8..77bf0bc8eb978 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java @@ -18,14 +18,16 @@ package org.apache.flink.test.recovery; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestartStrategyOptions; -import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; @@ -57,7 +59,9 @@ public void testFailedRunThenSuccessfulRun() throws Exception { try { // attempt 1 { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(4); Configuration configuration = new Configuration(); @@ -65,20 +69,25 @@ public void testFailedRunThenSuccessfulRun() throws Exception { env.configure(configuration, Thread.currentThread().getContextClassLoader()); try { - env.generateSequence(1, 10) - .rebalance() - .map(new FailingMapper1<>()) - .reduce(Long::sum) - .collect(); + CloseableIterator iterator = + env.fromSequence(1, 10) + .rebalance() + .map(new FailingMapper1<>()) + .fullWindowPartition() + .reduce(Long::sum) + .executeAndCollect(); + CollectionUtil.iteratorToList(iterator); fail("The program should have failed, but run successfully"); - } catch (JobExecutionException e) { + } catch (RuntimeException e) { // expected } } // attempt 2 { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(4); Configuration configuration = new Configuration(); @@ -86,11 +95,13 @@ public void testFailedRunThenSuccessfulRun() throws Exception { env.configure(configuration, Thread.currentThread().getContextClassLoader()); List resultCollection = - env.generateSequence(1, 10) - .rebalance() - .map(new FailingMapper1<>()) - .reduce((ReduceFunction) Long::sum) - .collect(); + CollectionUtil.iteratorToList( + env.fromSequence(1, 10) + .rebalance() + .map(new FailingMapper1<>()) + .fullWindowPartition() + .reduce((ReduceFunction) Long::sum) + .executeAndCollect()); long sum = 0; for (long l : resultCollection) { @@ -107,17 +118,20 @@ public void testFailedRunThenSuccessfulRun() throws Exception { @Test public void testRestart() throws Exception { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(4); // the default restart strategy should be taken List resultCollection = - env.generateSequence(1, 10) - .rebalance() - .map(new FailingMapper2<>()) - .reduce(Long::sum) - .collect(); + CollectionUtil.iteratorToList( + env.fromSequence(1, 10) + .rebalance() + .map(new FailingMapper2<>()) + .fullWindowPartition() + .reduce(Long::sum) + .executeAndCollect()); long sum = 0; for (long l : resultCollection) { @@ -132,16 +146,19 @@ public void testRestart() throws Exception { @Test public void testRestartMultipleTimes() throws Exception { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(4); List resultCollection = - env.generateSequence(1, 10) - .rebalance() - .map(new FailingMapper3<>()) - .reduce(Long::sum) - .collect(); + CollectionUtil.iteratorToList( + env.fromSequence(1, 10) + .rebalance() + .map(new FailingMapper3<>()) + .fullWindowPartition() + .reduce(Long::sum) + .executeAndCollect()); long sum = 0; for (long l : resultCollection) { @@ -162,7 +179,7 @@ private static class FailingMapper1 extends RichMapFunction { @Override public T map(T value) throws Exception { if (failuresBeforeSuccess > 0 - && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 1) { + && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { failuresBeforeSuccess--; throw new Exception("Test Failure"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java index 134e54c77051a..cf08bc35272f1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java @@ -18,43 +18,26 @@ package org.apache.flink.test.recovery; -import org.apache.flink.api.common.ExecutionMode; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; -import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; -import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +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.testutils.junit.extensions.parameterized.NoOpTestExtension; +import org.apache.flink.util.CollectionUtil; import org.junit.jupiter.api.extension.ExtendWith; import java.io.File; -import java.util.Arrays; -import java.util.Collection; import static org.assertj.core.api.Assertions.assertThat; /** Test the recovery of a simple batch program in the case of TaskManager process failure. */ -@ExtendWith(ParameterizedTestExtension.class) +@ExtendWith(NoOpTestExtension.class) public class TaskManagerProcessFailureBatchRecoveryITCase extends AbstractTaskManagerProcessFailureRecoveryTest { - - // -------------------------------------------------------------------------------------------- - // Parametrization (run pipelined and batch) - // -------------------------------------------------------------------------------------------- - - private final ExecutionMode executionMode; - - public TaskManagerProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) { - this.executionMode = executionMode; - } - - @Parameters(name = "executionMode={0}") - public static Collection executionMode() { - return Arrays.asList(new Object[][] {{ExecutionMode.PIPELINED}, {ExecutionMode.BATCH}}); - } - // -------------------------------------------------------------------------------------------- // Test the program // -------------------------------------------------------------------------------------------- @@ -62,15 +45,15 @@ public static Collection executionMode() { @Override public void testTaskManagerFailure(Configuration configuration, final File coordinateDir) throws Exception { - ExecutionEnvironment env = - ExecutionEnvironment.createRemoteEnvironment("localhost", 1337, configuration); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.createRemoteEnvironment( + "localhost", 1337, configuration); env.setParallelism(PARALLELISM); - env.getConfig().setExecutionMode(executionMode); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); final long numElements = 100000L; - final DataSet result = - env.generateSequence(1, numElements) - + final DataStream result = + env.fromSequence(1, numElements) // make sure every mapper is involved (no one is skipped because of lazy // split assignment) .rebalance() @@ -110,6 +93,8 @@ public Long map(Long value) throws Exception { return value; } }) + .setParallelism(PARALLELISM) + .windowAll(GlobalWindows.createWithEndOfStreamTrigger()) .reduce( new ReduceFunction() { @Override @@ -118,7 +103,10 @@ public Long reduce(Long value1, Long value2) { } }); - long sum = result.collect().get(0); + long sum = + CollectionUtil.iteratorToList(result.executeAndCollect()).stream() + .mapToLong(x -> x) + .sum(); assertThat(numElements * (numElements + 1L) / 2L).isEqualTo(sum); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/utils/TaskExecutorProcessEntryPoint.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/utils/TaskExecutorProcessEntryPoint.java index d2599c3b6a8f8..33fbd5ecbc2fd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/utils/TaskExecutorProcessEntryPoint.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/utils/TaskExecutorProcessEntryPoint.java @@ -18,12 +18,12 @@ package org.apache.flink.test.recovery.utils; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.util.ParameterTool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java deleted file mode 100644 index 02d98e89c5e75..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java +++ /dev/null @@ -1,74 +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.runtime; - -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.Tuple1; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -/** Test join with a slow source. */ -public class ConsumePipelinedAndBlockingResultITCase extends JavaProgramTestBaseJUnit4 { - - @Override - protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataSet> pipelinedSource = env.fromElements(new Tuple1(1L)); - - DataSet> slowBlockingSource = - env.generateSequence(0, 10) - .map( - new MapFunction>() { - @Override - public Tuple1 map(Long value) throws Exception { - Thread.sleep(200); - - return new Tuple1(value); - } - }); - - slowBlockingSource - .join(slowBlockingSource) - .where(0) - .equalTo(0) - .output(new DiscardingOutputFormat, Tuple1>>()); - - // Join the slow blocking and the pipelined source. This test should verify that this works - // w/o problems and the blocking result is not requested too early. - pipelinedSource - .join(slowBlockingSource) - .where(0) - .equalTo(0) - .output(new DiscardingOutputFormat, Tuple1>>()); - - env.execute("Consume one pipelined and one blocking result test job"); - } - - @Override - protected boolean skipCollectionExecution() { - // Skip collection execution as it is independent of the runtime environment functionality, - // which is under test. - return true; - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java index 19e5240021aee..a5d0215a959c0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java @@ -19,8 +19,7 @@ package org.apache.flink.test.runtime; import org.apache.flink.api.common.functions.FlatMapFunction; -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.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -29,9 +28,13 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcSystem; 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.windowing.assigners.GlobalWindows; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -83,13 +86,13 @@ private Configuration getConfiguration() { public void testClusterWithIPv6host() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(4); // get input data - DataSet text = env.fromElements(WordCountData.TEXT.split("\n")); + DataStream text = env.fromData(WordCountData.TEXT.split("\n")); - DataSet> counts = + DataStream> counts = text.flatMap( new FlatMapFunction>() { @Override @@ -105,10 +108,21 @@ public void flatMap( } } }) - .groupBy(0) - .sum(1); + .keyBy(x -> x.f0) + .window(GlobalWindows.createWithEndOfStreamTrigger()) + .reduce( + new ReduceFunction>() { + @Override + public Tuple2 reduce( + Tuple2 value1, + Tuple2 value2) + throws Exception { + return Tuple2.of(value1.f0, value1.f1 + value2.f1); + } + }); - List> result = counts.collect(); + List> result = + CollectionUtil.iteratorToList(counts.executeAndCollect()); TestBaseUtils.compareResultAsText(result, WordCountData.COUNTS_AS_TUPLES); } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java deleted file mode 100644 index 4dead6bc07016..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java +++ /dev/null @@ -1,76 +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.runtime; - -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.Tuple1; -import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; - -import org.junit.Rule; -import org.junit.rules.Timeout; - -/** - * Tests a join, which leads to a deadlock with large data sizes and PIPELINED-only execution. - * - * @see FLINK-1343 - */ -public class JoinDeadlockITCase extends JavaProgramTestBaseJUnit4 { - - protected String resultPath; - - @Rule public Timeout globalTimeout = new Timeout(120 * 1000); // Set timeout for deadlocks - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void testProgram() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet longs = env.generateSequence(0, 100000); - - DataSet> longT1 = longs.map(new TupleWrapper()); - DataSet> longT2 = longT1.project(0); - DataSet> longT3 = longs.map(new TupleWrapper()); - - longT2.join(longT3) - .where(0) - .equalTo(0) - .projectFirst(0) - .join(longT1) - .where(0) - .equalTo(0) - .projectFirst(0) - .writeAsText(resultPath); - - env.execute(); - } - - private static class TupleWrapper implements MapFunction> { - - @Override - public Tuple1 map(Long l) throws Exception { - return new Tuple1(l); - } - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java index 8a72849de3e48..28c57f521e32f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java @@ -21,10 +21,14 @@ import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.io.GenericInputFormat; import org.apache.flink.api.common.io.NonParallelInput; -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.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.test.util.JavaProgramTestBaseJUnit4; import org.apache.flink.util.Collector; @@ -52,12 +56,27 @@ protected void preSubmit() throws Exception { @Override protected void testProgram() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataSet> ds = + DataStreamSource> ds = env.createInput(new LargeJoinDataGeneratorInputFormat(1000000)); - ds.join(ds).where(0).equalTo(1).with(new Joiner()).writeAsText(resultPath); + ds.join(ds) + .where(x -> x.f0) + .equalTo(x -> x.f1) + .window(GlobalWindows.createWithEndOfStreamTrigger()) + .apply(new Joiner()) + .sinkTo( + FileSink.forRowFormat( + new Path(resultPath), + new SimpleStringEncoder< + Tuple5< + Integer, + Integer, + Integer, + String, + String>>()) + .build()); env.execute("Local Selfjoin Test Job"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java index d54735fc009ae..e2b498d0d14d7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java @@ -18,12 +18,12 @@ package org.apache.flink.test.runtime.entrypoint; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.functions.source.legacy.FileMonitoringFunction; import org.apache.flink.streaming.util.RestartStrategyUtils; +import org.apache.flink.util.ParameterTool; import java.io.File; import java.io.FileOutputStream; diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java index ab5a78d9598ac..4ef8c946c927f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java @@ -195,25 +195,6 @@ public void testScaleDownOnTaskManagerLoss() throws Exception { NUMBER_SLOTS_PER_TASK_MANAGER * NUMBER_SLOTS_PER_TASK_MANAGER); } - /** Test for FLINK-28274. */ - @Test - public void testContinuousFileMonitoringFunctionWithReactiveMode() throws Exception { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - final DataStream input = env.readTextFile(tempFolder.getRoot().getPath()); - input.sinkTo(new DiscardingSink<>()); - - final JobClient jobClient = env.executeAsync(); - - waitUntilParallelismForVertexReached( - miniClusterResource.getRestClusterClient(), jobClient.getJobID(), 1); - - // scale up to 2 TaskManagers: - miniClusterResource.getMiniCluster().startTaskManager(); - - waitUntilParallelismForVertexReached( - miniClusterResource.getRestClusterClient(), jobClient.getJobID(), 1); - } - private int getNumberOfConnectedTaskManagers() throws ExecutionException, InterruptedException { return miniClusterResource .getMiniCluster() diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java index d13441eca8baa..cb9a45f27dd8e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.CheckpointingMode; @@ -40,6 +39,7 @@ import org.apache.flink.streaming.util.StateBackendUtils; import org.apache.flink.test.state.operator.restore.ExecutionMode; import org.apache.flink.util.Collector; +import org.apache.flink.util.ParameterTool; import org.junit.Assert; diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java index 692530269e535..f7cf09604b681 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.CheckpointingMode; @@ -33,6 +32,7 @@ import org.apache.flink.streaming.util.RestartStrategyUtils; import org.apache.flink.streaming.util.StateBackendUtils; import org.apache.flink.test.state.operator.restore.ExecutionMode; +import org.apache.flink.util.ParameterTool; import org.junit.Assert; diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/functions/source/ContinuousFileReaderOperatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/functions/source/ContinuousFileReaderOperatorITCase.java index 502cb1107ede8..9d35168fcbfb2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/functions/source/ContinuousFileReaderOperatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/functions/source/ContinuousFileReaderOperatorITCase.java @@ -17,11 +17,14 @@ package org.apache.flink.test.streaming.api.functions.source; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.TextLineInputFormat; import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig; import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; @@ -47,7 +50,13 @@ public void testChainedOperatorsAreNotPrematurelyClosed() throws Exception { env.setParallelism(1); File input = temp.newFile("input"); FileUtils.write(input, "test", StandardCharsets.UTF_8); - DataStream stream = env.readTextFile(input.getAbsolutePath()); + + FileSource source = + FileSource.forRecordStreamFormat( + new TextLineInputFormat(), new Path(input.getAbsolutePath())) + .build(); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); final FileSink sink = FileSink.forRowFormat( new Path(temp.newFolder("output").getAbsolutePath()), diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java deleted file mode 100644 index b025dd180aacb..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java +++ /dev/null @@ -1,53 +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.streaming.api.outputformat; - -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.test.testdata.WordCountData; -import org.apache.flink.test.testfunctions.Tokenizer; -import org.apache.flink.test.util.AbstractTestBaseJUnit4; - -import org.junit.Test; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Integration tests for {@link org.apache.flink.api.java.io.CsvOutputFormat}. */ -public class CsvOutputFormatITCase extends AbstractTestBaseJUnit4 { - - @Test - public void testProgram() throws Exception { - String resultPath = getTempDirPath("result"); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream text = env.fromData(WordCountData.TEXT); - - DataStream> counts = text.flatMap(new Tokenizer()).keyBy(0).sum(1); - - counts.writeAsCsv(resultPath); - - env.execute("WriteAsCsvTest"); - - // Strip the parentheses from the expected text like output - compareResultsByLinesInMemory( - WordCountData.STREAMING_COUNTS_AS_TUPLES.replaceAll("[\\\\(\\\\)]", ""), - resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java deleted file mode 100644 index 5302b95d96923..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java +++ /dev/null @@ -1,50 +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.streaming.api.outputformat; - -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.test.testdata.WordCountData; -import org.apache.flink.test.testfunctions.Tokenizer; -import org.apache.flink.test.util.AbstractTestBaseJUnit4; - -import org.junit.Test; - -import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory; - -/** Integration tests for {@link org.apache.flink.api.java.io.TextOutputFormat}. */ -public class TextOutputFormatITCase extends AbstractTestBaseJUnit4 { - - @Test - public void testProgram() throws Exception { - String resultPath = getTempDirPath("result"); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream text = env.fromData(WordCountData.TEXT); - - DataStream> counts = text.flatMap(new Tokenizer()).keyBy(0).sum(1); - - counts.writeAsText(resultPath); - - env.execute("WriteAsTextTest"); - - compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java deleted file mode 100644 index eb75158dced06..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java +++ /dev/null @@ -1,244 +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.util; - -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.Utils; -import org.apache.flink.api.java.summarize.BooleanColumnSummary; -import org.apache.flink.api.java.summarize.NumericColumnSummary; -import org.apache.flink.api.java.summarize.StringColumnSummary; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple8; -import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.test.operators.util.CollectionDataSets; -import org.apache.flink.types.DoubleValue; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** Integration tests for {@link DataSetUtils}. */ -@RunWith(Parameterized.class) -public class DataSetUtilsITCase extends MultipleProgramsTestBaseJUnit4 { - - public DataSetUtilsITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testCountElementsPerPartition() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - long expectedSize = 100L; - DataSet numbers = env.generateSequence(0, expectedSize - 1); - - DataSet> ds = DataSetUtils.countElementsPerPartition(numbers); - - Assert.assertEquals(env.getParallelism(), ds.count()); - Assert.assertEquals(expectedSize, ds.sum(1).collect().get(0).f1.longValue()); - } - - @Test - public void testZipWithIndex() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - long expectedSize = 100L; - DataSet numbers = env.generateSequence(0, expectedSize - 1); - - List> result = - new ArrayList<>(DataSetUtils.zipWithIndex(numbers).collect()); - - Assert.assertEquals(expectedSize, result.size()); - // sort result by created index - Collections.sort( - result, - new Comparator>() { - @Override - public int compare(Tuple2 o1, Tuple2 o2) { - return o1.f0.compareTo(o2.f0); - } - }); - // test if index is consecutive - for (int i = 0; i < expectedSize; i++) { - Assert.assertEquals(i, result.get(i).f0.longValue()); - } - } - - @Test - public void testZipWithUniqueId() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - long expectedSize = 100L; - DataSet numbers = env.generateSequence(1L, expectedSize); - - DataSet ids = - DataSetUtils.zipWithUniqueId(numbers) - .map( - new MapFunction, Long>() { - @Override - public Long map(Tuple2 value) throws Exception { - return value.f0; - } - }); - - Set result = new HashSet<>(ids.collect()); - - Assert.assertEquals(expectedSize, result.size()); - } - - @Test - public void testIntegerDataSetChecksumHashCode() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet ds = CollectionDataSets.getIntegerDataSet(env); - - Utils.ChecksumHashCode checksum = DataSetUtils.checksumHashCode(ds); - Assert.assertEquals(checksum.getCount(), 15); - Assert.assertEquals(checksum.getChecksum(), 55); - } - - @Test - public void testSummarize() throws Exception { - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - List> data = - new ArrayList<>(); - data.add( - new Tuple8<>( - (short) 1, 1, 100L, 0.1f, 1.012376, "hello", false, new DoubleValue(50.0))); - data.add( - new Tuple8<>( - (short) 2, 2, 1000L, 0.2f, 2.003453, "hello", true, new DoubleValue(50.0))); - data.add( - new Tuple8<>( - (short) 4, - 10, - 10000L, - 0.2f, - 75.00005, - "null", - true, - new DoubleValue(50.0))); - data.add(new Tuple8<>((short) 10, 4, 100L, 0.9f, 79.5, "", true, new DoubleValue(50.0))); - data.add( - new Tuple8<>( - (short) 5, 5, 1000L, 0.2f, 10.0000001, "a", false, new DoubleValue(50.0))); - data.add( - new Tuple8<>( - (short) 6, - 6, - 10L, - 0.1f, - 0.0000000000023, - "", - true, - new DoubleValue(100.0))); - data.add( - new Tuple8<>( - (short) 7, - 7, - 1L, - 0.2f, - Double.POSITIVE_INFINITY, - "abcdefghijklmnop", - true, - new DoubleValue(100.0))); - data.add( - new Tuple8<>( - (short) 8, - 8, - -100L, - 0.001f, - Double.NaN, - "abcdefghi", - true, - new DoubleValue(100.0))); - - Collections.shuffle(data); - - DataSet> ds = - env.fromCollection(data); - - // call method under test - Tuple results = DataSetUtils.summarize(ds); - - Assert.assertEquals(8, results.getArity()); - - NumericColumnSummary col0Summary = results.getField(0); - Assert.assertEquals(8, col0Summary.getNonMissingCount()); - Assert.assertEquals(1, col0Summary.getMin().shortValue()); - Assert.assertEquals(10, col0Summary.getMax().shortValue()); - Assert.assertEquals(5.375, col0Summary.getMean().doubleValue(), 0.0); - - NumericColumnSummary col1Summary = results.getField(1); - Assert.assertEquals(1, col1Summary.getMin().intValue()); - Assert.assertEquals(10, col1Summary.getMax().intValue()); - Assert.assertEquals(5.375, col1Summary.getMean().doubleValue(), 0.0); - - NumericColumnSummary col2Summary = results.getField(2); - Assert.assertEquals(-100L, col2Summary.getMin().longValue()); - Assert.assertEquals(10000L, col2Summary.getMax().longValue()); - - NumericColumnSummary col3Summary = results.getField(3); - Assert.assertEquals(8, col3Summary.getTotalCount()); - Assert.assertEquals(0.001000, col3Summary.getMin().doubleValue(), 0.0000001); - Assert.assertEquals(0.89999999, col3Summary.getMax().doubleValue(), 0.0000001); - Assert.assertEquals( - 0.2376249988883501, col3Summary.getMean().doubleValue(), 0.000000000001); - Assert.assertEquals( - 0.0768965488108089, col3Summary.getVariance().doubleValue(), 0.00000001); - Assert.assertEquals( - 0.27730226975415995, - col3Summary.getStandardDeviation().doubleValue(), - 0.000000000001); - - NumericColumnSummary col4Summary = results.getField(4); - Assert.assertEquals(6, col4Summary.getNonMissingCount()); - Assert.assertEquals(2, col4Summary.getMissingCount()); - Assert.assertEquals(0.0000000000023, col4Summary.getMin().doubleValue(), 0.0); - Assert.assertEquals(79.5, col4Summary.getMax().doubleValue(), 0.000000000001); - - StringColumnSummary col5Summary = results.getField(5); - Assert.assertEquals(8, col5Summary.getTotalCount()); - Assert.assertEquals(0, col5Summary.getNullCount()); - Assert.assertEquals(8, col5Summary.getNonNullCount()); - Assert.assertEquals(2, col5Summary.getEmptyCount()); - Assert.assertEquals(0, col5Summary.getMinLength().intValue()); - Assert.assertEquals(16, col5Summary.getMaxLength().intValue()); - Assert.assertEquals(5.0, col5Summary.getMeanLength().doubleValue(), 0.0001); - - BooleanColumnSummary col6Summary = results.getField(6); - Assert.assertEquals(8, col6Summary.getTotalCount()); - Assert.assertEquals(2, col6Summary.getFalseCount()); - Assert.assertEquals(6, col6Summary.getTrueCount()); - Assert.assertEquals(0, col6Summary.getNullCount()); - - NumericColumnSummary col7Summary = results.getField(7); - Assert.assertEquals(100.0, col7Summary.getMax().doubleValue(), 0.00001); - Assert.assertEquals(50.0, col7Summary.getMin().doubleValue(), 0.00001); - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java b/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java index 0b791451ddd1f..b50bd2f0c0686 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java @@ -18,8 +18,8 @@ package org.apache.flink.test.util; -import org.apache.flink.api.java.io.TextOutputFormat.TextFormatter; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.legacy.io.TextOutputFormat.TextFormatter; import java.text.DecimalFormat; import java.text.DecimalFormatSymbols; diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index b22298a1efc4a..ce5bfca0df278 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -95,15 +95,6 @@ under the License. test - - org.apache.flink - flink-examples-batch - ${project.version} - jar - WordCount - test - - org.apache.flink flink-examples-streaming @@ -322,14 +313,6 @@ under the License. - - org.apache.flink - flink-examples-batch - jar - WordCount - true - BatchWordCount.jar - org.apache.flink flink-examples-streaming diff --git a/pom.xml b/pom.xml index 7d09d75a5c3ad..94bd1f42ee81c 100644 --- a/pom.xml +++ b/pom.xml @@ -72,7 +72,6 @@ under the License. flink-rpc flink-runtime flink-runtime-web - flink-optimizer flink-streaming-java flink-connectors flink-formats @@ -2396,6 +2395,10 @@ under the License. org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat org.apache.flink.api.java.typeutils.WritableTypeInfo + + org.apache.flink.api.java.utils.AbstractParameterTool + org.apache.flink.api.java.utils.ParameterTool org.apache.flink.table.api.typeutils.* diff --git a/tools/ci/stage.sh b/tools/ci/stage.sh index 940f9c82fd472..75d1d6c4b17a0 100755 --- a/tools/ci/stage.sh +++ b/tools/ci/stage.sh @@ -37,7 +37,6 @@ flink-state-backends/flink-statebackend-forst,\ flink-clients,\ flink-core,\ flink-java,\ -flink-optimizer,\ flink-rpc,\ flink-rpc/flink-rpc-core,\ flink-rpc/flink-rpc-akka,\