[ https://issues.apache.org/jira/browse/FLINK-10050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16622047#comment-16622047 ]
ASF GitHub Bot commented on FLINK-10050: ---------------------------------------- asfgit closed pull request #6646: [FLINK-10050] Support allowedLateness in CoGroupedStreams URL: https://github.com/apache/flink/pull/6646 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java index 55009e1b4cb..c8b552708c0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -40,6 +41,7 @@ import org.apache.flink.streaming.api.functions.windowing.WindowFunction; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; import org.apache.flink.streaming.api.windowing.evictors.Evictor; +import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.triggers.Trigger; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; @@ -183,7 +185,7 @@ public EqualTo equalTo(KeySelector<T2, KEY> keySelector, TypeInformation<KEY> ke */ @PublicEvolving public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) { - return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null); + return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null, null); } } } @@ -215,6 +217,12 @@ public EqualTo equalTo(KeySelector<T2, KEY> keySelector, TypeInformation<KEY> ke private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor; + @VisibleForTesting + Time allowedLateness; + + @VisibleForTesting + WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp; + protected WithWindow(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, @@ -222,7 +230,8 @@ protected WithWindow(DataStream<T1> input1, TypeInformation<KEY> keyType, WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner, Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger, - Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) { + Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor, + Time allowedLateness) { this.input1 = input1; this.input2 = input2; @@ -233,6 +242,8 @@ protected WithWindow(DataStream<T1> input1, this.windowAssigner = windowAssigner; this.trigger = trigger; this.evictor = evictor; + + this.allowedLateness = allowedLateness; } /** @@ -241,7 +252,7 @@ protected WithWindow(DataStream<T1> input1, @PublicEvolving public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) { return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, - windowAssigner, newTrigger, evictor); + windowAssigner, newTrigger, evictor, allowedLateness); } /** @@ -254,7 +265,18 @@ protected WithWindow(DataStream<T1> input1, @PublicEvolving public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) { return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, - windowAssigner, trigger, newEvictor); + windowAssigner, trigger, newEvictor, allowedLateness); + } + + /** + * Sets the time by which elements are allowed to be late. + * @see WindowedStream#allowedLateness(Time) + */ + @PublicEvolving + public WithWindow<T1, T2, KEY, W> allowedLateness(Time newLateness) { + Preconditions.checkNotNull(newLateness, "The allowed lateness must not be null"); + this.allowedLateness = newLateness; + return this; } /** @@ -321,9 +343,7 @@ protected WithWindow(DataStream<T1> input1, DataStream<TaggedUnion<T1, T2>> unionStream = taggedInput1.union(taggedInput2); // we explicitly create the keyed stream to manually pass the key type information in - WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp = - new KeyedStream<TaggedUnion<T1, T2>, KEY>(unionStream, unionKeySelector, keyType) - .window(windowAssigner); + windowOp = new KeyedStream<TaggedUnion<T1, T2>, KEY>(unionStream, unionKeySelector, keyType).window(windowAssigner); if (trigger != null) { windowOp.trigger(trigger); @@ -331,6 +351,9 @@ protected WithWindow(DataStream<T1> input1, if (evictor != null) { windowOp.evictor(evictor); } + if (allowedLateness != null) { + windowOp.allowedLateness(allowedLateness); + } return windowOp.apply(new CoGroupWindowFunction<T1, T2, T, KEY, W>(function), resultType); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java index bb67c091423..0755c9fea5a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java @@ -19,6 +19,7 @@ import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.JoinFunction; @@ -29,9 +30,11 @@ import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; import org.apache.flink.streaming.api.windowing.evictors.Evictor; +import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.triggers.Trigger; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; import static java.util.Objects.requireNonNull; @@ -166,7 +169,7 @@ public EqualTo equalTo(KeySelector<T2, KEY> keySelector, TypeInformation<KEY> ke */ @PublicEvolving public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) { - return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null); + return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null, null); } } } @@ -198,6 +201,12 @@ public EqualTo equalTo(KeySelector<T2, KEY> keySelector, TypeInformation<KEY> ke private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor; + @VisibleForTesting + Time allowedLateness; + + @VisibleForTesting + CoGroupedStreams.WithWindow<T1, T2, KEY, W> coGroupedWindow; + @PublicEvolving protected WithWindow(DataStream<T1> input1, DataStream<T2> input2, @@ -206,7 +215,8 @@ protected WithWindow(DataStream<T1> input1, TypeInformation<KEY> keyType, WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner, Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger, - Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) { + Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor, + Time allowedLateness) { this.input1 = requireNonNull(input1); this.input2 = requireNonNull(input2); @@ -219,6 +229,8 @@ protected WithWindow(DataStream<T1> input1, this.trigger = trigger; this.evictor = evictor; + + this.allowedLateness = allowedLateness; } /** @@ -227,7 +239,7 @@ protected WithWindow(DataStream<T1> input1, @PublicEvolving public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) { return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, - windowAssigner, newTrigger, evictor); + windowAssigner, newTrigger, evictor, allowedLateness); } /** @@ -239,7 +251,18 @@ protected WithWindow(DataStream<T1> input1, @PublicEvolving public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) { return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, - windowAssigner, trigger, newEvictor); + windowAssigner, trigger, newEvictor, allowedLateness); + } + + /** + * Sets the time by which elements are allowed to be late. + * @see WindowedStream#allowedLateness(Time) + */ + @PublicEvolving + public WithWindow<T1, T2, KEY, W> allowedLateness(Time newLateness) { + Preconditions.checkNotNull(newLateness, "The allowed lateness must not be null"); + this.allowedLateness = newLateness; + return this; } /** @@ -295,13 +318,18 @@ protected WithWindow(DataStream<T1> input1, //clean the closure function = input1.getExecutionEnvironment().clean(function); - return input1.coGroup(input2) - .where(keySelector1) - .equalTo(keySelector2) - .window(windowAssigner) - .trigger(trigger) - .evictor(evictor) - .apply(new FlatJoinCoGroupFunction<>(function), resultType); + coGroupedWindow = input1.coGroup(input2) + .where(keySelector1) + .equalTo(keySelector2) + .window(windowAssigner) + .trigger(trigger) + .evictor(evictor); + + if (allowedLateness != null) { + coGroupedWindow.allowedLateness(allowedLateness); + } + + return coGroupedWindow.apply(new FlatJoinCoGroupFunction<>(function), resultType); } @@ -376,14 +404,18 @@ protected WithWindow(DataStream<T1> input1, //clean the closure function = input1.getExecutionEnvironment().clean(function); - return input1.coGroup(input2) - .where(keySelector1) - .equalTo(keySelector2) - .window(windowAssigner) - .trigger(trigger) - .evictor(evictor) - .apply(new JoinCoGroupFunction<>(function), resultType); + coGroupedWindow = input1.coGroup(input2) + .where(keySelector1) + .equalTo(keySelector2) + .window(windowAssigner) + .trigger(trigger) + .evictor(evictor); + + if (allowedLateness != null) { + coGroupedWindow.allowedLateness(allowedLateness); + } + return coGroupedWindow.apply(new JoinCoGroupFunction<>(function), resultType); } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index 1f09b737080..17c8eb9abdc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.functions.FoldFunction; import org.apache.flink.api.common.functions.Function; @@ -113,7 +114,8 @@ private Evictor<? super T, ? super W> evictor; /** The user-specified allowed lateness. */ - private long allowedLateness = 0L; + @VisibleForTesting + long allowedLateness = 0L; /** * Side output {@code OutputTag} for late data. If no tag is set late data will simply be diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/CoGroupedStreamsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/CoGroupedStreamsTest.java new file mode 100644 index 00000000000..565f4a67f90 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/CoGroupedStreamsTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.datastream; + +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +/** + * Unit test for {@link CoGroupedStreams}. + */ +public class CoGroupedStreamsTest { + private DataStream<String> dataStream1; + private DataStream<String> dataStream2; + private KeySelector<String, String> keySelector; + private TumblingEventTimeWindows tsAssigner; + private CoGroupFunction<String, String, String> coGroupFunction; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Before + public void setUp() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + dataStream1 = env.fromElements("a1", "a2", "a3"); + dataStream2 = env.fromElements("a1", "a2"); + keySelector = element -> element; + tsAssigner = TumblingEventTimeWindows.of(Time.milliseconds(1)); + coGroupFunction = (CoGroupFunction<String, String, String>) (first, second, out) -> out.collect(""); + } + + @Test + public void testNullAllowedLatenessThrows() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("The allowed lateness must not be null"); + + dataStream1 + .coGroup(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(null); + } + + @Test + public void testDelegateToCoGrouped() { + Time lateness = Time.milliseconds(42); + + CoGroupedStreams.WithWindow<String, String, String, TimeWindow> withLateness = dataStream1 + .coGroup(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(lateness); + + withLateness.apply(coGroupFunction, BasicTypeInfo.STRING_TYPE_INFO); + + Assert.assertEquals(lateness.toMilliseconds(), withLateness.windowOp.allowedLateness); + } + + @Test + public void testSetAllowedLateness() { + Time lateness = Time.milliseconds(42); + + CoGroupedStreams.WithWindow<String, String, String, TimeWindow> withLateness = dataStream1 + .coGroup(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(lateness); + + Assert.assertEquals(lateness, withLateness.allowedLateness); + } + +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/JoinedStreamsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/JoinedStreamsTest.java new file mode 100644 index 00000000000..34e8b0de3b8 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/JoinedStreamsTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.datastream; + +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +/** + * Unit test for {@link JoinedStreams}. + */ +public class JoinedStreamsTest { + private DataStream<String> dataStream1; + private DataStream<String> dataStream2; + private KeySelector<String, String> keySelector; + private TumblingEventTimeWindows tsAssigner; + private JoinFunction<String, String, String> joinFunction; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Before + public void setUp() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + dataStream1 = env.fromElements("a1", "a2", "a3"); + dataStream2 = env.fromElements("a1", "a2"); + keySelector = element -> element; + tsAssigner = TumblingEventTimeWindows.of(Time.milliseconds(1)); + joinFunction = (first, second) -> first + second; + } + + @Test + public void testNullAllowedLatenessThrows() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("The allowed lateness must not be null"); + + dataStream1 + .join(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(null); + } + + @Test + public void testDelegateToCoGrouped() { + Time lateness = Time.milliseconds(42); + + JoinedStreams.WithWindow<String, String, String, TimeWindow> withLateness = dataStream1 + .join(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(lateness); + + withLateness.apply(joinFunction, BasicTypeInfo.STRING_TYPE_INFO); + + Assert.assertEquals(lateness, withLateness.coGroupedWindow.allowedLateness); + } + + @Test + public void testSetAllowedLateness() { + Time lateness = Time.milliseconds(42); + + JoinedStreams.WithWindow<String, String, String, TimeWindow> withLateness = dataStream1 + .join(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(lateness); + + Assert.assertEquals(lateness, withLateness.allowedLateness); + } +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala index 101d3588c31..16877a3ce74 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.streaming.api.datastream.{CoGroupedStreams => JavaCoGroupedStreams} import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner import org.apache.flink.streaming.api.windowing.evictors.Evictor +import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.Trigger import org.apache.flink.streaming.api.windowing.windows.Window import org.apache.flink.util.Collector @@ -112,7 +113,7 @@ class CoGroupedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { throw new UnsupportedOperationException( "You first need to specify KeySelectors for both inputs using where() and equalTo().") } - new WithWindow[W](clean(assigner), null, null) + new WithWindow[W](clean(assigner), null, null, null) } /** @@ -125,7 +126,8 @@ class CoGroupedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { class WithWindow[W <: Window]( windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W], trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W], - evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) { + evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W], + val allowedLateness: Time) { /** * Sets the [[Trigger]] that should be used to trigger window emission. @@ -133,7 +135,7 @@ class CoGroupedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { @PublicEvolving def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) : WithWindow[W] = { - new WithWindow[W](windowAssigner, newTrigger, evictor) + new WithWindow[W](windowAssigner, newTrigger, evictor, allowedLateness) } /** @@ -147,7 +149,17 @@ class CoGroupedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { def evictor( newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) : WithWindow[W] = { - new WithWindow[W](windowAssigner, trigger, newEvictor) + new WithWindow[W](windowAssigner, trigger, newEvictor, allowedLateness) + } + + /** + * Sets the time by which elements are allowed to be late. + * Delegates to [[WindowedStream#allowedLateness(Time)]] + */ + @PublicEvolving + def allowedLateness(newLateness: Time): WithWindow[W] = { + require(newLateness != null, "The allowed lateness must not be null") + new WithWindow[W](windowAssigner, trigger, evictor, newLateness) } /** @@ -196,13 +208,19 @@ class CoGroupedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { val coGroup = new JavaCoGroupedStreams[T1, T2](input1.javaStream, input2.javaStream) - asScalaStream(coGroup + val withWindow = coGroup .where(keySelector1) .equalTo(keySelector2) .window(windowAssigner) .trigger(trigger) .evictor(evictor) - .apply(clean(function), implicitly[TypeInformation[T]])) + val withLateness = if (allowedLateness != null) { + withWindow.allowedLateness(allowedLateness) + } else { + withWindow + } + + asScalaStream(withLateness.apply(clean(function), implicitly[TypeInformation[T]])) } } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala index 93b5cc885b4..6d3f91d8179 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.streaming.api.datastream.{JoinedStreams => JavaJoinedStreams, CoGroupedStreams => JavaCoGroupedStreams} import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner import org.apache.flink.streaming.api.windowing.evictors.Evictor +import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.Trigger import org.apache.flink.streaming.api.windowing.windows.Window import org.apache.flink.util.Collector @@ -110,7 +111,7 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { "You first need to specify KeySelectors for both inputs using where() and equalTo().") } - new WithWindow[W](clean(assigner), null, null) + new WithWindow[W](clean(assigner), null, null, null) } /** @@ -122,7 +123,8 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { class WithWindow[W <: Window]( windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W], trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W], - evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) { + evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W], + val allowedLateness: Time) { /** * Sets the [[Trigger]] that should be used to trigger window emission. @@ -130,7 +132,7 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { @PublicEvolving def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) : WithWindow[W] = { - new WithWindow[W](windowAssigner, newTrigger, evictor) + new WithWindow[W](windowAssigner, newTrigger, evictor, allowedLateness) } /** @@ -142,7 +144,17 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { @PublicEvolving def evictor(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) : WithWindow[W] = { - new WithWindow[W](windowAssigner, trigger, newEvictor) + new WithWindow[W](windowAssigner, trigger, newEvictor, allowedLateness) + } + + /** + * Sets the time by which elements are allowed to be late. + * Delegates to [[WindowedStream#allowedLateness(Time)]] + */ + @PublicEvolving + def allowedLateness(newLateness: Time): WithWindow[W] = { + require(newLateness != null, "The allowed lateness must not be null") + new WithWindow[W](windowAssigner, trigger, evictor, newLateness) } /** @@ -185,13 +197,19 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { val join = new JavaJoinedStreams[T1, T2](input1.javaStream, input2.javaStream) - asScalaStream(join + val withWindow = join .where(keySelector1) .equalTo(keySelector2) .window(windowAssigner) .trigger(trigger) .evictor(evictor) - .apply(clean(function), implicitly[TypeInformation[T]])) + val withLateness = if (allowedLateness != null) { + withWindow.allowedLateness(allowedLateness) + } else { + withWindow + } + + asScalaStream(withLateness.apply(clean(function), implicitly[TypeInformation[T]])) } /** @@ -202,12 +220,19 @@ class JoinedStreams[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) { val join = new JavaJoinedStreams[T1, T2](input1.javaStream, input2.javaStream) - asScalaStream(join + val withWindow = join .where(keySelector1) .equalTo(keySelector2) .window(windowAssigner) .trigger(trigger) .evictor(evictor) + val withLateness = if (allowedLateness != null) { + withWindow.allowedLateness(allowedLateness) + } else { + withWindow + } + + asScalaStream(withLateness .apply(clean(function), implicitly[TypeInformation[T]])) } } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupedStreamsTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupedStreamsTest.scala new file mode 100644 index 00000000000..cee1a315fce --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupedStreamsTest.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala + +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows +import org.apache.flink.streaming.api.windowing.time.Time +import org.junit.{Assert, Test} + +/** + * Unit test for [[org.apache.flink.streaming.api.scala.CoGroupedStreams]] + */ +class CoGroupedStreamsTest { + private val env = StreamExecutionEnvironment.getExecutionEnvironment + + private val dataStream1 = env.fromElements("a1", "a2", "a3") + private val dataStream2 = env.fromElements("a1", "a2") + private val keySelector = (s: String) => s + private val tsAssigner = TumblingEventTimeWindows.of(Time.milliseconds(1)) + + @Test(expected = classOf[IllegalArgumentException]) + def testNullAllowedLatenessThrows(): Unit = { + dataStream1.coGroup(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(null) + } + + @Test + def testSetAllowedLateness(): Unit = { + val lateness = Time.milliseconds(42) + val withLateness = dataStream1.coGroup(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(lateness) + Assert.assertEquals(lateness, withLateness.allowedLateness) + } + +} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/JoinedStreamsTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/JoinedStreamsTest.scala new file mode 100644 index 00000000000..8d64f6eb91e --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/JoinedStreamsTest.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala + +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows +import org.apache.flink.streaming.api.windowing.time.Time +import org.junit.{Assert, Test} + +/** + * Unit test for [[org.apache.flink.streaming.api.scala.JoinedStreams]] + */ +class JoinedStreamsTest { + private val env = StreamExecutionEnvironment.getExecutionEnvironment + + private val dataStream1 = env.fromElements("a1", "a2", "a3") + private val dataStream2 = env.fromElements("a1", "a2") + private val keySelector = (s: String) => s + private val tsAssigner = TumblingEventTimeWindows.of(Time.milliseconds(1)) + + @Test(expected = classOf[IllegalArgumentException]) + def testNullAllowedLatenessThrows(): Unit = { + dataStream1.join(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(null) + } + + @Test + def testSetAllowedLateness(): Unit = { + val lateness = Time.milliseconds(42) + val withLateness = dataStream1.join(dataStream2) + .where(keySelector) + .equalTo(keySelector) + .window(tsAssigner) + .allowedLateness(lateness) + Assert.assertEquals(lateness, withLateness.allowedLateness) + } +} ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Support 'allowedLateness' in CoGroupedStreams > --------------------------------------------- > > Key: FLINK-10050 > URL: https://issues.apache.org/jira/browse/FLINK-10050 > Project: Flink > Issue Type: Improvement > Components: Streaming > Affects Versions: 1.5.1, 1.6.0 > Reporter: eugen yushin > Priority: Major > Labels: pull-request-available, ready-to-commit, windows > > WindowedStream has a support of 'allowedLateness' feature, while > CoGroupedStreams are not. At the mean time, WindowedStream is an inner part > of CoGroupedStreams and all main functionality (like evictor/trigger/...) is > simply delegated to WindowedStream. > There's no chance to operate with late arriving data from previous steps in > cogroups (and joins). Consider the following flow: > a. read data from source1 -> aggregate data with allowed lateness > b. read data from source2 -> aggregate data with allowed lateness > c. cogroup/join streams a and b, and compare aggregated values > Step c doesn't accept any late data from steps a/b due to lack of > `allowedLateness` API call in CoGroupedStreams.java. > Scope: add method `WithWindow.allowedLateness` to Java API > (flink-streaming-java) and extend scala API (flink-streaming-scala). -- This message was sent by Atlassian JIRA (v7.6.3#76005)