From bfb8dc45feebaf52b1e8e02bd119837cb1c11207 Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Wed, 10 Jan 2018 15:50:00 +0000 Subject: [PATCH 1/7] [FLINK-8384] [streaming] Dynamic Gap Session Window Assigner --- docs/dev/stream/operators/windows.md | 55 ++++- .../DynamicEventTimeSessionWindows.java | 101 +++++++++ .../DynamicProcessingTimeSessionWindows.java | 102 +++++++++ .../SessionWindowTimeGapExtractor.java | 30 +++ .../triggers/TypedEventTimeTrigger.java | 89 ++++++++ .../triggers/TypedProcessingTimeTrigger.java | 80 +++++++ .../DynamicEventTimeSessionWindowsTest.java | 200 +++++++++++++++++ ...namicProcessingTimeSessionWindowsTest.java | 203 ++++++++++++++++++ .../windowing/TypedEventTimeTriggerTest.java | 156 ++++++++++++++ .../TypedProcessingTimeTriggerTest.java | 137 ++++++++++++ .../windowing/WindowOperatorTest.java | 178 +++++++++++++++ 11 files changed, 1323 insertions(+), 8 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 0327d0629bf50..836ce44ea1922 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -281,9 +281,9 @@ For example, in China you would have to specify an offset of `Time.hours(-8)`. The *session windows* assigner groups elements by sessions of activity. Session windows do not overlap and do not have a fixed start and end time, in contrast to *tumbling windows* and *sliding windows*. Instead a session window closes when it does not receive elements for a certain period of time, *i.e.*, when a gap of -inactivity occurred. A session window assigner is configured with the *session gap* which -defines how long is the required period of inactivity. When this period expires, the current session closes -and subsequent elements are assigned to a new session window. +inactivity occurred. A session window assigner can be configured with either a static *session gap* or with a +*session gap extractor* function which defines how long the period of inactivity is. When this period expires, +the current session closes and subsequent elements are assigned to a new session window. @@ -294,17 +294,33 @@ The following code snippets show how to use session windows. {% highlight java %} DataStream input = ...; -// event-time session windows +// event-time session windows with static gap input .keyBy() .window(EventTimeSessionWindows.withGap(Time.minutes(10))) .(); + +// event-time session windows with dynamic gap +input + .keyBy() + .window(DynamicEventTimeSessionWindows.withDynamicGap((element, timestamp, context) -> { + // determine and return session gap + })) + .(); -// processing-time session windows +// processing-time session windows with static gap input .keyBy() .window(ProcessingTimeSessionWindows.withGap(Time.minutes(10))) .(); + +// processing-time session windows with dynamic gap +input + .keyBy() + .window(DynamicProcessingTimeSessionWindows.withDynamicGap((element, timestamp, context) -> { + // determine and return session gap + })) + .(); {% endhighlight %} @@ -312,24 +328,47 @@ input {% highlight scala %} val input: DataStream[T] = ... -// event-time session windows +// event-time session windows with static gap input .keyBy() .window(EventTimeSessionWindows.withGap(Time.minutes(10))) .() -// processing-time session windows +// event-time session windows with dynamic gap +input + .keyBy() + .window(DynamicEventTimeSessionWindows.withDynamicGap(new SessionWindowTimeGapExtractor[String] { + override def extract(element: String, ts: Long, ctx: WindowAssigner.WindowAssignerContext): Long = { + // determine and return session gap + } + })) + .() + +// processing-time session windows with static gap input .keyBy() .window(ProcessingTimeSessionWindows.withGap(Time.minutes(10))) .() + + +// processing-time session windows with dynamic gap +input + .keyBy() + .window(DynamicProcessingTimeSessionWindows.withDynamicGap(new SessionWindowTimeGapExtractor[String] { + override def extract(element: String, ts: Long, ctx: WindowAssigner.WindowAssignerContext): Long = { + // determine and return session gap + } + })) + .() {% endhighlight %} -Time intervals can be specified by using one of `Time.milliseconds(x)`, `Time.seconds(x)`, +Static gaps can be specified by using one of `Time.milliseconds(x)`, `Time.seconds(x)`, `Time.minutes(x)`, and so on. +Dynamic gaps are specified by implementing the `SessionWindowTimeGapExtractor` interface. + Attention Since session windows do not have a fixed start and end, they are evaluated differently than tumbling and sliding windows. Internally, a session window operator creates a new window for each arriving record and merges windows together if their are closer to each other diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java new file mode 100644 index 0000000000000..c31b03aca5c77 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.windowing.assigners; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TypedEventTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +import java.util.Collection; +import java.util.Collections; + +/** + * A {@link WindowAssigner} that windows elements into sessions based on the timestamp of the + * elements. Windows cannot overlap. + * + *

For example, in order to window into windows with a dynamic time gap: + *

 {@code
+ * DataStream> in = ...;
+ * KeyedStream> keyed = in.keyBy(...);
+ * WindowedStream, String, TimeWindows> windowed =
+ *   keyed.window(DynamicEventTimeSessionWindows.withDynamicGap({@link SessionWindowTimeGapExtractor }));
+ * } 
+ * + * @param The type of the input elements + */ +public class DynamicEventTimeSessionWindows extends MergingWindowAssigner { + private static final long serialVersionUID = 1L; + + protected SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor; + + protected DynamicEventTimeSessionWindows(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { + this.sessionWindowTimeGapExtractor = sessionWindowTimeGapExtractor; + } + + @Override + public Collection assignWindows(T element, long timestamp, WindowAssignerContext context) { + long sessionTimeout = sessionWindowTimeGapExtractor.extract(element, timestamp, context); + if (sessionTimeout <= 0) { + throw new IllegalArgumentException("Dynamic session time gap must satisfy 0 < gap"); + } + return Collections.singletonList(new TimeWindow(timestamp, timestamp + sessionTimeout)); + } + + @Override + public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { + return TypedEventTimeTrigger.create(); + } + + @Override + public String toString() { + return "DynamicEventTimeSessionWindows()"; + } + + /** + * Creates a new {@code SessionWindows} {@link WindowAssigner} that assigns + * elements to sessions based on the element timestamp. + * + * @param sessionWindowTimeGapExtractor The extractor to use to extract the time gap from the input elements + * @return The policy. + */ + public static DynamicEventTimeSessionWindows withDynamicGap(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { + return new DynamicEventTimeSessionWindows<>(sessionWindowTimeGapExtractor); + } + + @Override + public TypeSerializer getWindowSerializer(ExecutionConfig executionConfig) { + return new TimeWindow.Serializer(); + } + + @Override + public boolean isEventTime() { + return true; + } + + /** + * Merge overlapping {@link TimeWindow}s. + */ + public void mergeWindows(Collection windows, MergeCallback c) { + TimeWindow.mergeWindows(windows, c); + } + +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java new file mode 100644 index 0000000000000..11d36a2d3e4b8 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.windowing.assigners; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +import java.util.Collection; +import java.util.Collections; + +/** + * A {@link WindowAssigner} that windows elements into sessions based on the current processing + * time. Windows cannot overlap. + * + *

For example, in order to window into windows with a dynamic time gap: + *

 {@code
+ * DataStream> in = ...;
+ * KeyedStream> keyed = in.keyBy(...);
+ * WindowedStream, String, TimeWindows> windowed =
+ *   keyed.window(DynamicProcessingTimeSessionWindows.withDynamicGap({@link SessionWindowTimeGapExtractor }));
+ * } 
+ * + * @param The type of the input elements + */ +public class DynamicProcessingTimeSessionWindows extends MergingWindowAssigner { + private static final long serialVersionUID = 1L; + + protected SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor; + + protected DynamicProcessingTimeSessionWindows(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { + this.sessionWindowTimeGapExtractor = sessionWindowTimeGapExtractor; + } + + @Override + public Collection assignWindows(T element, long timestamp, WindowAssignerContext context) { + long currentProcessingTime = context.getCurrentProcessingTime(); + long sessionTimeout = sessionWindowTimeGapExtractor.extract(element, timestamp, context); + if (sessionTimeout <= 0) { + throw new IllegalArgumentException("Dynamic session time gap must satisfy 0 < gap"); + } + return Collections.singletonList(new TimeWindow(currentProcessingTime, currentProcessingTime + sessionTimeout)); + } + + @Override + public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { + return TypedProcessingTimeTrigger.create(); + } + + @Override + public String toString() { + return "DynamicProcessingTimeSessionWindows()"; + } + + /** + * Creates a new {@code SessionWindows} {@link WindowAssigner} that assigns + * elements to sessions based on the element timestamp. + * + * @param sessionWindowTimeGapExtractor The extractor to use to extract the time gap from the input elements + * @return The policy. + */ + public static DynamicProcessingTimeSessionWindows withDynamicGap(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { + return new DynamicProcessingTimeSessionWindows<>(sessionWindowTimeGapExtractor); + } + + @Override + public TypeSerializer getWindowSerializer(ExecutionConfig executionConfig) { + return new TimeWindow.Serializer(); + } + + @Override + public boolean isEventTime() { + return false; + } + + /** + * Merge overlapping {@link TimeWindow}s. + */ + public void mergeWindows(Collection windows, MergeCallback c) { + TimeWindow.mergeWindows(windows, c); + } + +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java new file mode 100644 index 0000000000000..efece7ec9b6e1 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.windowing.assigners; + +import java.io.Serializable; + +/** + * A {@code SessionWindowTimeGapExtractor} extracts session time gaps for Dynamic Session Window Assigners. + * + * @param The type of elements that this {@code SessionWindowTimeGapExtractor} can extract session time gaps from. + */ +public interface SessionWindowTimeGapExtractor extends Serializable { + long extract(T element, long timestamp, WindowAssigner.WindowAssignerContext context); +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java new file mode 100644 index 0000000000000..b786f5b40caf3 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.windowing.triggers; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +/** + * A {@link Trigger} that fires once the current system time passes the end of the window + * to which a pane belongs. + * + * @param The type of the input elements + */ +@PublicEvolving +public class TypedEventTimeTrigger extends Trigger { + private static final long serialVersionUID = 1L; + + private TypedEventTimeTrigger() {} + + @Override + public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception { + if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { + // if the watermark is already past the window fire immediately + return TriggerResult.FIRE; + } else { + ctx.registerEventTimeTimer(window.maxTimestamp()); + return TriggerResult.CONTINUE; + } + } + + @Override + public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) { + return time == window.maxTimestamp() ? + TriggerResult.FIRE : + TriggerResult.CONTINUE; + } + + @Override + public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) throws Exception { + return TriggerResult.CONTINUE; + } + + @Override + public void clear(TimeWindow window, TriggerContext ctx) throws Exception { + ctx.deleteEventTimeTimer(window.maxTimestamp()); + } + + @Override + public boolean canMerge() { + return true; + } + + @Override + public void onMerge(TimeWindow window, + OnMergeContext ctx) { + ctx.registerEventTimeTimer(window.maxTimestamp()); + } + + @Override + public String toString() { + return "TypedEventTimeTrigger()"; + } + + /** + * Creates an event-time trigger that fires once the watermark passes the end of the window. + * + *

Once the trigger fires all elements are discarded. Elements that arrive late immediately + * trigger window evaluation with just this one element. + */ + public static TypedEventTimeTrigger create() { + return new TypedEventTimeTrigger<>(); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java new file mode 100644 index 0000000000000..b5380541de658 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.windowing.triggers; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +/** + * A {@link Trigger} that fires once the current system time passes the end of the window + * to which a pane belongs. + * + * @param The type of the input elements + */ +@PublicEvolving +public class TypedProcessingTimeTrigger extends Trigger { + private static final long serialVersionUID = 1L; + + private TypedProcessingTimeTrigger() {} + + @Override + public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) { + ctx.registerProcessingTimeTimer(window.maxTimestamp()); + return TriggerResult.CONTINUE; + } + + @Override + public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) throws Exception { + return TriggerResult.CONTINUE; + } + + @Override + public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) { + return TriggerResult.FIRE; + } + + @Override + public void clear(TimeWindow window, TriggerContext ctx) throws Exception { + ctx.deleteProcessingTimeTimer(window.maxTimestamp()); + } + + @Override + public boolean canMerge() { + return true; + } + + @Override + public void onMerge(TimeWindow window, + OnMergeContext ctx) { + ctx.registerProcessingTimeTimer(window.maxTimestamp()); + } + + @Override + public String toString() { + return "TypedProcessingTimeTrigger()"; + } + + /** + * Creates a new trigger that fires once system time passes the end of the window. + */ + public static TypedProcessingTimeTrigger create() { + return new TypedProcessingTimeTrigger<>(); + } + +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java new file mode 100644 index 0000000000000..b9395c7697ffd --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.operators.windowing; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.DynamicEventTimeSessionWindows; +import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner; +import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; +import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; +import org.apache.flink.streaming.api.windowing.triggers.TypedEventTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.junit.Test; +import org.mockito.Matchers; + +import java.util.Collection; + +import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyCollection; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for {@link DynamicEventTimeSessionWindows}. + */ +public class DynamicEventTimeSessionWindowsTest extends TestLogger { + + @Test + public void testWindowAssignment() { + + WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(eq("gap5000"), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(eq("gap4000"), anyLong(), any())).thenReturn(4000L); + when(extractor.extract(eq("gap9000"), anyLong(), any())).thenReturn(9000L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + + assertThat(assigner.assignWindows("gap5000", 0L, mockContext), contains(timeWindow(0, 5000))); + assertThat(assigner.assignWindows("gap4000", 4999L, mockContext), contains(timeWindow(4999, 8999))); + assertThat(assigner.assignWindows("gap9000", 5000L, mockContext), contains(timeWindow(5000, 14000))); + } + + @Test + public void testMergeSinglePointWindow() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 0)), callback); + + verify(callback, never()).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testMergeSingleWindow() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 1)), callback); + + verify(callback, never()).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testMergeConsecutiveWindows() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows( + Lists.newArrayList( + new TimeWindow(0, 1), + new TimeWindow(1, 2), + new TimeWindow(2, 3), + new TimeWindow(4, 5), + new TimeWindow(5, 6)), + callback); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(0, 1), new TimeWindow(1, 2), new TimeWindow(2, 3))), + eq(new TimeWindow(0, 3))); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(4, 5), new TimeWindow(5, 6))), + eq(new TimeWindow(4, 6))); + + verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testMergeCoveringWindow() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows( + Lists.newArrayList( + new TimeWindow(1, 1), + new TimeWindow(0, 2), + new TimeWindow(4, 7), + new TimeWindow(5, 6)), + callback); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(1, 1), new TimeWindow(0, 2))), + eq(new TimeWindow(0, 2))); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(5, 6), new TimeWindow(4, 7))), + eq(new TimeWindow(4, 7))); + + verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testInvalidParameters() { + WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); + try { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(-1L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); + + fail("should fail"); + } catch (IllegalArgumentException e) { + assertThat(e.toString(), containsString("0 < gap")); + } + + try { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(0L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); + + fail("should fail"); + } catch (IllegalArgumentException e) { + assertThat(e.toString(), containsString("0 < gap")); + } + + } + + @Test + public void testProperties() { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); + + assertTrue(assigner.isEventTime()); + assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig())); + assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(TypedEventTimeTrigger.class)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java new file mode 100644 index 0000000000000..03744b36434a9 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.operators.windowing; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.DynamicProcessingTimeSessionWindows; +import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner; +import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; +import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; +import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.junit.Test; +import org.mockito.Matchers; + +import java.util.Collection; + +import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyCollection; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for {@link DynamicProcessingTimeSessionWindows}. + */ +public class DynamicProcessingTimeSessionWindowsTest extends TestLogger { + + @Test + public void testWindowAssignment() { + + WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(eq("gap5000"), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(eq("gap4000"), anyLong(), any())).thenReturn(4000L); + when(extractor.extract(eq("gap9000"), anyLong(), any())).thenReturn(9000L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + + when(mockContext.getCurrentProcessingTime()).thenReturn(0L); + assertThat(assigner.assignWindows("gap5000", Long.MIN_VALUE, mockContext), contains(timeWindow(0, 5000))); + + when(mockContext.getCurrentProcessingTime()).thenReturn(4999L); + assertThat(assigner.assignWindows("gap4000", Long.MIN_VALUE, mockContext), contains(timeWindow(4999, 8999))); + + when(mockContext.getCurrentProcessingTime()).thenReturn(5000L); + assertThat(assigner.assignWindows("gap9000", Long.MIN_VALUE, mockContext), contains(timeWindow(5000, 14000))); + } + + @Test + public void testMergeSinglePointWindow() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 0)), callback); + + verify(callback, never()).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testMergeSingleWindow() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 1)), callback); + + verify(callback, never()).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testMergeConsecutiveWindows() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows( + Lists.newArrayList( + new TimeWindow(0, 1), + new TimeWindow(1, 2), + new TimeWindow(2, 3), + new TimeWindow(4, 5), + new TimeWindow(5, 6)), + callback); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(0, 1), new TimeWindow(1, 2), new TimeWindow(2, 3))), + eq(new TimeWindow(0, 3))); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(4, 5), new TimeWindow(5, 6))), + eq(new TimeWindow(4, 6))); + + verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testMergeCoveringWindow() { + MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + + assigner.mergeWindows( + Lists.newArrayList( + new TimeWindow(1, 1), + new TimeWindow(0, 2), + new TimeWindow(4, 7), + new TimeWindow(5, 6)), + callback); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(1, 1), new TimeWindow(0, 2))), + eq(new TimeWindow(0, 2))); + + verify(callback, times(1)).merge( + (Collection) argThat(containsInAnyOrder(new TimeWindow(5, 6), new TimeWindow(4, 7))), + eq(new TimeWindow(4, 7))); + + verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject()); + } + + @Test + public void testInvalidParameters() { + WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); + try { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(-1L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); + fail("should fail"); + } catch (IllegalArgumentException e) { + assertThat(e.toString(), containsString("0 < gap")); + } + + try { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(-1L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); + fail("should fail"); + } catch (IllegalArgumentException e) { + assertThat(e.toString(), containsString("0 < gap")); + } + + } + + @Test + public void testProperties() { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + + DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); + + assertFalse(assigner.isEventTime()); + assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig())); + assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(TypedProcessingTimeTrigger.class)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java new file mode 100644 index 0000000000000..6747e086b2237 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.operators.windowing; + +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; +import org.apache.flink.streaming.api.windowing.triggers.TypedEventTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link TypedEventTimeTrigger}. + */ +public class TypedEventTimeTriggerTest { + + /** + * Verify that state of separate windows does not leak into other windows. + */ + @Test + public void testWindowSeparationAndFiring() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); + + // inject several elements + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(2, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); + + assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(2, new TimeWindow(0, 2))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); + + assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(4, new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers()); + } + + /** + * Verify that late elements trigger immediately and also that we don't set a timer + * for those. + */ + @Test + public void testLateElementTriggersImmediately() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); + + testHarness.advanceWatermark(2); + + assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers()); + } + + + /** + * Verify that clear() does not leak across windows. + */ + @Test + public void testClear() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); + + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(2, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); + + testHarness.clearTriggerState(new TimeWindow(2, 4)); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); + assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); + + testHarness.clearTriggerState(new TimeWindow(0, 2)); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers()); + } + + @Test + public void testMergingWindows() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); + + assertTrue(TypedEventTimeTrigger.create().canMerge()); + + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(2, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); + + testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numEventTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); + assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); + assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 4))); + + assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(4, new TimeWindow(0, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers()); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java new file mode 100644 index 0000000000000..1ed176e51eead --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.operators.windowing; + +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; +import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link TypedProcessingTimeTrigger}. + */ +public class TypedProcessingTimeTriggerTest { + + /** + * Verify that state of separate windows does not leak into other windows. + */ + @Test + public void testWindowSeparationAndFiring() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedProcessingTimeTrigger.create(), new TimeWindow.Serializer()); + + // inject several elements + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numEventTimeTimers()); + assertEquals(2, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); + + assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(2, new TimeWindow(0, 2))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); + + assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(4, new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers()); + } + + /** + * Verify that clear() does not leak across windows. + */ + @Test + public void testClear() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedProcessingTimeTrigger.create(), new TimeWindow.Serializer()); + + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numEventTimeTimers()); + assertEquals(2, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); + + testHarness.clearTriggerState(new TimeWindow(2, 4)); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); + assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); + + testHarness.clearTriggerState(new TimeWindow(0, 2)); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + } + + @Test + public void testMergingWindows() throws Exception { + TriggerTestHarness testHarness = + new TriggerTestHarness<>(TypedProcessingTimeTrigger.create(), new TimeWindow.Serializer()); + + assertTrue(TypedProcessingTimeTrigger.create().canMerge()); + + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); + assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numEventTimeTimers()); + assertEquals(2, testHarness.numProcessingTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); + + testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numEventTimeTimers()); + assertEquals(1, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); + assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); + assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 4))); + + assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(4, new TimeWindow(0, 4))); + + assertEquals(0, testHarness.numStateEntries()); + assertEquals(0, testHarness.numProcessingTimeTimers()); + assertEquals(0, testHarness.numEventTimeTimers()); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index c03207e7f1634..f72c5db912594 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -37,9 +37,12 @@ import org.apache.flink.streaming.api.functions.windowing.WindowFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.DynamicEventTimeSessionWindows; +import org.apache.flink.streaming.api.windowing.assigners.DynamicProcessingTimeSessionWindows; import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; import org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows; +import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; @@ -86,6 +89,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Tests for {@link WindowOperator}. @@ -1253,6 +1260,177 @@ public void testProcessingTimeSessionWindows() throws Throwable { testHarness.close(); } + @Test + @SuppressWarnings("unchecked") + public void testDynamicEventTimeSessionWindows() throws Exception { + closeCalled.set(0); + + SessionWindowTimeGapExtractor> extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(Tuple2.class), anyLong(), any())).thenAnswer(invocation -> { + Tuple2 element = (Tuple2) invocation.getArguments()[0]; + switch (element.f0) { + case "key1": + return 3000; + case "key2": + switch (element.f1) { + case 10: + return 1000; + default: + return 2000; + } + default: + return 0; + } + }); + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple3, TimeWindow> operator = new WindowOperator<>( + DynamicEventTimeSessionWindows.withDynamicGap(extractor), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new SessionWindowFunction()), + EventTimeTrigger.create(), + 0, + null /* late data output tag */); + + OneInputStreamOperatorTestHarness, Tuple3> testHarness = + createTestHarness(operator); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + // test different gaps for different keys + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 10)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 5000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 5), 6000)); + + testHarness.processWatermark(new Watermark(8999)); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-3", 10L, 3010L), 3009)); + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-9", 5000L, 8000L), 7999)); + expectedOutput.add(new Watermark(8999)); + + // test gap when it produces an end time before current timeout + // the furthest timeout is respected + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 9000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 10000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 10), 10500)); + + testHarness.processWatermark(new Watermark(12999)); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-13", 9000L, 12000L), 11999)); + expectedOutput.add(new Watermark(12999)); + + // test gap when it produces an end time after current timeout + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 10), 13000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 10), 13500)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14000)); + + testHarness.processWatermark(new Watermark(16999)); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-21", 13000L, 16000L), 15999)); + expectedOutput.add(new Watermark(16999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); + + testHarness.close(); + } + + @Test + @SuppressWarnings("unchecked") + public void testDynamicProcessingTimeSessionWindows() throws Exception { + closeCalled.set(0); + + SessionWindowTimeGapExtractor> extractor = mock(SessionWindowTimeGapExtractor.class); + when(extractor.extract(any(Tuple2.class), anyLong(), any())).thenAnswer(invocation -> { + Tuple2 element = (Tuple2) invocation.getArguments()[0]; + switch (element.f0) { + case "key1": + return 3000; + case "key2": + switch (element.f1) { + case 10: + return 1000; + default: + return 2000; + } + default: + return 0; + } + }); + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple3, TimeWindow> operator = new WindowOperator<>( + DynamicProcessingTimeSessionWindows.withDynamicGap(extractor), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new SessionWindowFunction()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */); + + OneInputStreamOperatorTestHarness, Tuple3> testHarness = + createTestHarness(operator); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + // test different gaps for different keys + testHarness.setProcessingTime(10); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 10)); + + testHarness.setProcessingTime(5000); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 5000)); + testHarness.setProcessingTime(6000); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 5), 6000)); + testHarness.setProcessingTime(8999); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-3", 10L, 3010L), 3009)); + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-9", 5000L, 8000L), 7999)); + + // test gap when it produces an end time before current timeout + // the furthest timeout is respected + testHarness.setProcessingTime(9000); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 9000)); + testHarness.setProcessingTime(10000); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 10000)); + testHarness.setProcessingTime(10500); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 10), 10500)); + testHarness.setProcessingTime(10500); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-13", 9000L, 12000L), 11999)); + + // test gap when it produces an end time after current timeout + testHarness.setProcessingTime(13000); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 10), 13000)); + testHarness.setProcessingTime(13500); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 10), 13500)); + testHarness.setProcessingTime(14000); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14000)); + testHarness.setProcessingTime(16999); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-21", 13000L, 16000L), 15999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); + + testHarness.close(); + } + @Test public void testLateness() throws Exception { final int windowSize = 2; From 5619885ce8af5c9e92c654f8d4474feab5a64c1b Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Thu, 18 Jan 2018 09:56:02 +0000 Subject: [PATCH 2/7] [FLINK-8384] [streaming] Session gap extractor to only take the element --- .../DynamicEventTimeSessionWindows.java | 2 +- .../DynamicProcessingTimeSessionWindows.java | 2 +- .../SessionWindowTimeGapExtractor.java | 2 +- .../DynamicEventTimeSessionWindowsTest.java | 21 +++++++++---------- ...namicProcessingTimeSessionWindowsTest.java | 21 +++++++++---------- .../windowing/WindowOperatorTest.java | 5 ++--- 6 files changed, 25 insertions(+), 28 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java index c31b03aca5c77..03f0f321ca6d1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java @@ -53,7 +53,7 @@ protected DynamicEventTimeSessionWindows(SessionWindowTimeGapExtractor sessio @Override public Collection assignWindows(T element, long timestamp, WindowAssignerContext context) { - long sessionTimeout = sessionWindowTimeGapExtractor.extract(element, timestamp, context); + long sessionTimeout = sessionWindowTimeGapExtractor.extract(element); if (sessionTimeout <= 0) { throw new IllegalArgumentException("Dynamic session time gap must satisfy 0 < gap"); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java index 11d36a2d3e4b8..75c9df48a2d0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java @@ -54,7 +54,7 @@ protected DynamicProcessingTimeSessionWindows(SessionWindowTimeGapExtractor s @Override public Collection assignWindows(T element, long timestamp, WindowAssignerContext context) { long currentProcessingTime = context.getCurrentProcessingTime(); - long sessionTimeout = sessionWindowTimeGapExtractor.extract(element, timestamp, context); + long sessionTimeout = sessionWindowTimeGapExtractor.extract(element); if (sessionTimeout <= 0) { throw new IllegalArgumentException("Dynamic session time gap must satisfy 0 < gap"); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java index efece7ec9b6e1..8b66cd4d486db 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java @@ -26,5 +26,5 @@ * @param The type of elements that this {@code SessionWindowTimeGapExtractor} can extract session time gaps from. */ public interface SessionWindowTimeGapExtractor extends Serializable { - long extract(T element, long timestamp, WindowAssigner.WindowAssignerContext context); + long extract(T element); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java index b9395c7697ffd..9ea81d5fca806 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java @@ -46,7 +46,6 @@ import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyCollection; -import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.argThat; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; @@ -65,9 +64,9 @@ public void testWindowAssignment() { WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(eq("gap5000"), anyLong(), any())).thenReturn(5000L); - when(extractor.extract(eq("gap4000"), anyLong(), any())).thenReturn(4000L); - when(extractor.extract(eq("gap9000"), anyLong(), any())).thenReturn(9000L); + when(extractor.extract(eq("gap5000"))).thenReturn(5000L); + when(extractor.extract(eq("gap4000"))).thenReturn(4000L); + when(extractor.extract(eq("gap9000"))).thenReturn(9000L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); @@ -80,7 +79,7 @@ public void testWindowAssignment() { public void testMergeSinglePointWindow() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); @@ -93,7 +92,7 @@ public void testMergeSinglePointWindow() { public void testMergeSingleWindow() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); @@ -106,7 +105,7 @@ public void testMergeSingleWindow() { public void testMergeConsecutiveWindows() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); @@ -134,7 +133,7 @@ public void testMergeConsecutiveWindows() { public void testMergeCoveringWindow() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); @@ -162,7 +161,7 @@ public void testInvalidParameters() { WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); try { SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(-1L); + when(extractor.extract(any())).thenReturn(-1L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); @@ -174,7 +173,7 @@ public void testInvalidParameters() { try { SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(0L); + when(extractor.extract(any())).thenReturn(0L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); @@ -189,7 +188,7 @@ public void testInvalidParameters() { @Test public void testProperties() { SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicEventTimeSessionWindows assigner = DynamicEventTimeSessionWindows.withDynamicGap(extractor); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java index 03744b36434a9..d3c1347db2fdf 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java @@ -46,7 +46,6 @@ import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyCollection; -import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.argThat; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; @@ -65,9 +64,9 @@ public void testWindowAssignment() { WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(eq("gap5000"), anyLong(), any())).thenReturn(5000L); - when(extractor.extract(eq("gap4000"), anyLong(), any())).thenReturn(4000L); - when(extractor.extract(eq("gap9000"), anyLong(), any())).thenReturn(9000L); + when(extractor.extract(eq("gap5000"))).thenReturn(5000L); + when(extractor.extract(eq("gap4000"))).thenReturn(4000L); + when(extractor.extract(eq("gap9000"))).thenReturn(9000L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); @@ -85,7 +84,7 @@ public void testWindowAssignment() { public void testMergeSinglePointWindow() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); @@ -98,7 +97,7 @@ public void testMergeSinglePointWindow() { public void testMergeSingleWindow() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); @@ -111,7 +110,7 @@ public void testMergeSingleWindow() { public void testMergeConsecutiveWindows() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); @@ -139,7 +138,7 @@ public void testMergeConsecutiveWindows() { public void testMergeCoveringWindow() { MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class); SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); @@ -167,7 +166,7 @@ public void testInvalidParameters() { WindowAssigner.WindowAssignerContext mockContext = mock(WindowAssigner.WindowAssignerContext.class); try { SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(-1L); + when(extractor.extract(any())).thenReturn(-1L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); @@ -178,7 +177,7 @@ public void testInvalidParameters() { try { SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(-1L); + when(extractor.extract(any())).thenReturn(-1L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); assigner.assignWindows(Lists.newArrayList(new Object()), 1, mockContext); @@ -192,7 +191,7 @@ public void testInvalidParameters() { @Test public void testProperties() { SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(), anyLong(), any())).thenReturn(5000L); + when(extractor.extract(any())).thenReturn(5000L); DynamicProcessingTimeSessionWindows assigner = DynamicProcessingTimeSessionWindows.withDynamicGap(extractor); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index f72c5db912594..f85b7fbd03d34 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -90,7 +90,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -1266,7 +1265,7 @@ public void testDynamicEventTimeSessionWindows() throws Exception { closeCalled.set(0); SessionWindowTimeGapExtractor> extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(Tuple2.class), anyLong(), any())).thenAnswer(invocation -> { + when(extractor.extract(any(Tuple2.class))).thenAnswer(invocation -> { Tuple2 element = (Tuple2) invocation.getArguments()[0]; switch (element.f0) { case "key1": @@ -1350,7 +1349,7 @@ public void testDynamicProcessingTimeSessionWindows() throws Exception { closeCalled.set(0); SessionWindowTimeGapExtractor> extractor = mock(SessionWindowTimeGapExtractor.class); - when(extractor.extract(any(Tuple2.class), anyLong(), any())).thenAnswer(invocation -> { + when(extractor.extract(any(Tuple2.class))).thenAnswer(invocation -> { Tuple2 element = (Tuple2) invocation.getArguments()[0]; switch (element.f0) { case "key1": From f00aa7953b5852ea25348173b51e286d841f33d3 Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Thu, 25 Jan 2018 09:58:22 +0000 Subject: [PATCH 3/7] [FLINK-8384] [streaming] Add generic type to event and processing time triggers --- .../DynamicEventTimeSessionWindows.java | 4 +- .../DynamicProcessingTimeSessionWindows.java | 4 +- .../windowing/triggers/EventTimeTrigger.java | 10 +- .../triggers/ProcessingTimeTrigger.java | 10 +- .../triggers/TypedEventTimeTrigger.java | 89 ---------- .../triggers/TypedProcessingTimeTrigger.java | 80 --------- .../DynamicEventTimeSessionWindowsTest.java | 4 +- ...namicProcessingTimeSessionWindowsTest.java | 4 +- .../windowing/TypedEventTimeTriggerTest.java | 156 ------------------ .../TypedProcessingTimeTriggerTest.java | 137 --------------- .../api/scala/AllWindowTranslationTest.scala | 70 ++++---- .../api/scala/TimeWindowTranslationTest.scala | 6 +- .../api/scala/WindowTranslationTest.scala | 82 ++++----- 13 files changed, 99 insertions(+), 557 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java index 03f0f321ca6d1..e3d6fe0f84f33 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger; import org.apache.flink.streaming.api.windowing.triggers.Trigger; -import org.apache.flink.streaming.api.windowing.triggers.TypedEventTimeTrigger; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import java.util.Collection; @@ -62,7 +62,7 @@ public Collection assignWindows(T element, long timestamp, WindowAss @Override public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { - return TypedEventTimeTrigger.create(); + return EventTimeTrigger.create(); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java index 75c9df48a2d0b..95f08fbaf552a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger; import org.apache.flink.streaming.api.windowing.triggers.Trigger; -import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import java.util.Collection; @@ -63,7 +63,7 @@ public Collection assignWindows(T element, long timestamp, WindowAss @Override public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { - return TypedProcessingTimeTrigger.create(); + return ProcessingTimeTrigger.create(); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java index 2f8f16f6b1d40..55f276ec90ec9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java @@ -26,15 +26,17 @@ * to which a pane belongs. * * @see org.apache.flink.streaming.api.watermark.Watermark + * + * @param The type of the input elements */ @PublicEvolving -public class EventTimeTrigger extends Trigger { +public class EventTimeTrigger extends Trigger { private static final long serialVersionUID = 1L; private EventTimeTrigger() {} @Override - public TriggerResult onElement(Object element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception { + public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception { if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { // if the watermark is already past the window fire immediately return TriggerResult.FIRE; @@ -83,7 +85,7 @@ public String toString() { *

Once the trigger fires all elements are discarded. Elements that arrive late immediately * trigger window evaluation with just this one element. */ - public static EventTimeTrigger create() { - return new EventTimeTrigger(); + public static EventTimeTrigger create() { + return new EventTimeTrigger<>(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java index cd7869e3d78fb..ebe9b07a4c8c7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java @@ -24,15 +24,17 @@ /** * A {@link Trigger} that fires once the current system time passes the end of the window * to which a pane belongs. + * + * @param The type of the input elements */ @PublicEvolving -public class ProcessingTimeTrigger extends Trigger { +public class ProcessingTimeTrigger extends Trigger { private static final long serialVersionUID = 1L; private ProcessingTimeTrigger() {} @Override - public TriggerResult onElement(Object element, long timestamp, TimeWindow window, TriggerContext ctx) { + public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) { ctx.registerProcessingTimeTimer(window.maxTimestamp()); return TriggerResult.CONTINUE; } @@ -71,8 +73,8 @@ public String toString() { /** * Creates a new trigger that fires once system time passes the end of the window. */ - public static ProcessingTimeTrigger create() { - return new ProcessingTimeTrigger(); + public static ProcessingTimeTrigger create() { + return new ProcessingTimeTrigger<>(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.java deleted file mode 100644 index b786f5b40caf3..0000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedEventTimeTrigger.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.streaming.api.windowing.triggers; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; - -/** - * A {@link Trigger} that fires once the current system time passes the end of the window - * to which a pane belongs. - * - * @param The type of the input elements - */ -@PublicEvolving -public class TypedEventTimeTrigger extends Trigger { - private static final long serialVersionUID = 1L; - - private TypedEventTimeTrigger() {} - - @Override - public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception { - if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { - // if the watermark is already past the window fire immediately - return TriggerResult.FIRE; - } else { - ctx.registerEventTimeTimer(window.maxTimestamp()); - return TriggerResult.CONTINUE; - } - } - - @Override - public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) { - return time == window.maxTimestamp() ? - TriggerResult.FIRE : - TriggerResult.CONTINUE; - } - - @Override - public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) throws Exception { - return TriggerResult.CONTINUE; - } - - @Override - public void clear(TimeWindow window, TriggerContext ctx) throws Exception { - ctx.deleteEventTimeTimer(window.maxTimestamp()); - } - - @Override - public boolean canMerge() { - return true; - } - - @Override - public void onMerge(TimeWindow window, - OnMergeContext ctx) { - ctx.registerEventTimeTimer(window.maxTimestamp()); - } - - @Override - public String toString() { - return "TypedEventTimeTrigger()"; - } - - /** - * Creates an event-time trigger that fires once the watermark passes the end of the window. - * - *

Once the trigger fires all elements are discarded. Elements that arrive late immediately - * trigger window evaluation with just this one element. - */ - public static TypedEventTimeTrigger create() { - return new TypedEventTimeTrigger<>(); - } -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.java deleted file mode 100644 index b5380541de658..0000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/TypedProcessingTimeTrigger.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.streaming.api.windowing.triggers; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; - -/** - * A {@link Trigger} that fires once the current system time passes the end of the window - * to which a pane belongs. - * - * @param The type of the input elements - */ -@PublicEvolving -public class TypedProcessingTimeTrigger extends Trigger { - private static final long serialVersionUID = 1L; - - private TypedProcessingTimeTrigger() {} - - @Override - public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) { - ctx.registerProcessingTimeTimer(window.maxTimestamp()); - return TriggerResult.CONTINUE; - } - - @Override - public TriggerResult onEventTime(long time, TimeWindow window, TriggerContext ctx) throws Exception { - return TriggerResult.CONTINUE; - } - - @Override - public TriggerResult onProcessingTime(long time, TimeWindow window, TriggerContext ctx) { - return TriggerResult.FIRE; - } - - @Override - public void clear(TimeWindow window, TriggerContext ctx) throws Exception { - ctx.deleteProcessingTimeTimer(window.maxTimestamp()); - } - - @Override - public boolean canMerge() { - return true; - } - - @Override - public void onMerge(TimeWindow window, - OnMergeContext ctx) { - ctx.registerProcessingTimeTimer(window.maxTimestamp()); - } - - @Override - public String toString() { - return "TypedProcessingTimeTrigger()"; - } - - /** - * Creates a new trigger that fires once system time passes the end of the window. - */ - public static TypedProcessingTimeTrigger create() { - return new TypedProcessingTimeTrigger<>(); - } - -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java index 9ea81d5fca806..d048dad7a4663 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicEventTimeSessionWindowsTest.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner; import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; -import org.apache.flink.streaming.api.windowing.triggers.TypedEventTimeTrigger; +import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.util.TestLogger; @@ -194,6 +194,6 @@ public void testProperties() { assertTrue(assigner.isEventTime()); assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig())); - assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(TypedEventTimeTrigger.class)); + assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(EventTimeTrigger.class)); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java index d3c1347db2fdf..a6bcedc607507 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/DynamicProcessingTimeSessionWindowsTest.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner; import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; -import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; +import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.util.TestLogger; @@ -197,6 +197,6 @@ public void testProperties() { assertFalse(assigner.isEventTime()); assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig())); - assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(TypedProcessingTimeTrigger.class)); + assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(ProcessingTimeTrigger.class)); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java deleted file mode 100644 index 6747e086b2237..0000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedEventTimeTriggerTest.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.operators.windowing; - -import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; -import org.apache.flink.streaming.api.windowing.triggers.TypedEventTimeTrigger; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; - -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * Tests for {@link TypedEventTimeTrigger}. - */ -public class TypedEventTimeTriggerTest { - - /** - * Verify that state of separate windows does not leak into other windows. - */ - @Test - public void testWindowSeparationAndFiring() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); - - // inject several elements - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(2, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); - - assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(2, new TimeWindow(0, 2))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); - - assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(4, new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers()); - } - - /** - * Verify that late elements trigger immediately and also that we don't set a timer - * for those. - */ - @Test - public void testLateElementTriggersImmediately() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); - - testHarness.advanceWatermark(2); - - assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers()); - } - - - /** - * Verify that clear() does not leak across windows. - */ - @Test - public void testClear() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); - - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(2, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); - - testHarness.clearTriggerState(new TimeWindow(2, 4)); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); - assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); - - testHarness.clearTriggerState(new TimeWindow(0, 2)); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers()); - } - - @Test - public void testMergingWindows() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedEventTimeTrigger.create(), new TimeWindow.Serializer()); - - assertTrue(TypedEventTimeTrigger.create().canMerge()); - - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(2, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); - - testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numEventTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2))); - assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(2, 4))); - assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 4))); - - assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(4, new TimeWindow(0, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers()); - } -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java deleted file mode 100644 index 1ed176e51eead..0000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TypedProcessingTimeTriggerTest.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.operators.windowing; - -import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; -import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; - -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * Tests for {@link TypedProcessingTimeTrigger}. - */ -public class TypedProcessingTimeTriggerTest { - - /** - * Verify that state of separate windows does not leak into other windows. - */ - @Test - public void testWindowSeparationAndFiring() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedProcessingTimeTrigger.create(), new TimeWindow.Serializer()); - - // inject several elements - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numEventTimeTimers()); - assertEquals(2, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); - - assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(2, new TimeWindow(0, 2))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); - - assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(4, new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers()); - } - - /** - * Verify that clear() does not leak across windows. - */ - @Test - public void testClear() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedProcessingTimeTrigger.create(), new TimeWindow.Serializer()); - - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numEventTimeTimers()); - assertEquals(2, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); - - testHarness.clearTriggerState(new TimeWindow(2, 4)); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); - assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); - - testHarness.clearTriggerState(new TimeWindow(0, 2)); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - } - - @Test - public void testMergingWindows() throws Exception { - TriggerTestHarness testHarness = - new TriggerTestHarness<>(TypedProcessingTimeTrigger.create(), new TimeWindow.Serializer()); - - assertTrue(TypedProcessingTimeTrigger.create().canMerge()); - - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(0, 2))); - assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numEventTimeTimers()); - assertEquals(2, testHarness.numProcessingTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); - - testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numEventTimeTimers()); - assertEquals(1, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2))); - assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4))); - assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 4))); - - assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(4, new TimeWindow(0, 4))); - - assertEquals(0, testHarness.numStateEntries()); - assertEquals(0, testHarness.numProcessingTimeTimers()); - assertEquals(0, testHarness.numEventTimeTimers()); - } -} diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala index e8d5a1221bbb9..66767b1a0cd02 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala @@ -197,7 +197,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[EventTimeSessionWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -235,7 +235,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -268,7 +268,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -301,7 +301,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -340,7 +340,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -379,7 +379,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -419,7 +419,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -458,7 +458,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -497,7 +497,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -532,7 +532,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -569,7 +569,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -602,7 +602,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -635,7 +635,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -668,7 +668,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -701,7 +701,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -734,7 +734,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -771,7 +771,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -808,7 +808,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -841,7 +841,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -874,7 +874,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -916,7 +916,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -957,7 +957,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -998,7 +998,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1039,7 +1039,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1081,7 +1081,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1119,7 +1119,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1162,7 +1162,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1201,7 +1201,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1240,7 +1240,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1279,7 +1279,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1314,7 +1314,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1498,7 +1498,7 @@ class AllWindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1534,7 +1534,7 @@ class AllWindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1579,7 +1579,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1620,7 +1620,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala index db0fb71c2f256..85b3d10ff4845 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala @@ -106,7 +106,7 @@ class TimeWindowTranslationTest extends AbstractTestBase { val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _, _]] - assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator1.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) } @@ -132,7 +132,7 @@ class TimeWindowTranslationTest extends AbstractTestBase { val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _, _]] - assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator1.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) } @@ -164,7 +164,7 @@ class TimeWindowTranslationTest extends AbstractTestBase { val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _, _]] - assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator1.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala index 916884ff5ffc2..95cf97fbca860 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala @@ -202,7 +202,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[EventTimeSessionWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -240,7 +240,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -274,7 +274,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -308,7 +308,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -349,7 +349,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -390,7 +390,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -432,7 +432,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -474,7 +474,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -515,7 +515,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -558,7 +558,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -595,7 +595,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -633,7 +633,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -667,7 +667,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -701,7 +701,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -735,7 +735,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -769,7 +769,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -803,7 +803,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -840,7 +840,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -878,7 +878,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -912,7 +912,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -946,7 +946,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -990,7 +990,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1033,7 +1033,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1076,7 +1076,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1119,7 +1119,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1163,7 +1163,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1208,7 +1208,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1247,7 +1247,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1292,7 +1292,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1333,7 +1333,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1374,7 +1374,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1415,7 +1415,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1451,7 +1451,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1643,7 +1643,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1680,7 +1680,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1716,7 +1716,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1751,7 +1751,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1787,7 +1787,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1828,7 +1828,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1876,7 +1876,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1919,7 +1919,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) From 52f1e1e708e2197b5c0b78c1288628101179dfe9 Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Thu, 1 Feb 2018 10:59:03 +0000 Subject: [PATCH 4/7] [FLINK-8384] [streaming] Cast triggers to types --- .../DynamicEventTimeSessionWindows.java | 3 +- .../DynamicProcessingTimeSessionWindows.java | 3 +- .../windowing/triggers/EventTimeTrigger.java | 10 +-- .../triggers/ProcessingTimeTrigger.java | 10 +-- .../api/scala/AllWindowTranslationTest.scala | 70 ++++++++-------- .../api/scala/TimeWindowTranslationTest.scala | 6 +- .../api/scala/WindowTranslationTest.scala | 82 +++++++++---------- 7 files changed, 91 insertions(+), 93 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java index e3d6fe0f84f33..0f5c3ee852121 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java @@ -60,9 +60,10 @@ public Collection assignWindows(T element, long timestamp, WindowAss return Collections.singletonList(new TimeWindow(timestamp, timestamp + sessionTimeout)); } + @SuppressWarnings("unchecked") @Override public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { - return EventTimeTrigger.create(); + return (Trigger) EventTimeTrigger.create(); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java index 95f08fbaf552a..05be06c9e81cf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java @@ -61,9 +61,10 @@ public Collection assignWindows(T element, long timestamp, WindowAss return Collections.singletonList(new TimeWindow(currentProcessingTime, currentProcessingTime + sessionTimeout)); } + @SuppressWarnings("unchecked") @Override public Trigger getDefaultTrigger(StreamExecutionEnvironment env) { - return ProcessingTimeTrigger.create(); + return (Trigger) ProcessingTimeTrigger.create(); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java index 55f276ec90ec9..2f8f16f6b1d40 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java @@ -26,17 +26,15 @@ * to which a pane belongs. * * @see org.apache.flink.streaming.api.watermark.Watermark - * - * @param The type of the input elements */ @PublicEvolving -public class EventTimeTrigger extends Trigger { +public class EventTimeTrigger extends Trigger { private static final long serialVersionUID = 1L; private EventTimeTrigger() {} @Override - public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception { + public TriggerResult onElement(Object element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception { if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { // if the watermark is already past the window fire immediately return TriggerResult.FIRE; @@ -85,7 +83,7 @@ public String toString() { *

Once the trigger fires all elements are discarded. Elements that arrive late immediately * trigger window evaluation with just this one element. */ - public static EventTimeTrigger create() { - return new EventTimeTrigger<>(); + public static EventTimeTrigger create() { + return new EventTimeTrigger(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java index ebe9b07a4c8c7..cd7869e3d78fb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java @@ -24,17 +24,15 @@ /** * A {@link Trigger} that fires once the current system time passes the end of the window * to which a pane belongs. - * - * @param The type of the input elements */ @PublicEvolving -public class ProcessingTimeTrigger extends Trigger { +public class ProcessingTimeTrigger extends Trigger { private static final long serialVersionUID = 1L; private ProcessingTimeTrigger() {} @Override - public TriggerResult onElement(T element, long timestamp, TimeWindow window, TriggerContext ctx) { + public TriggerResult onElement(Object element, long timestamp, TimeWindow window, TriggerContext ctx) { ctx.registerProcessingTimeTimer(window.maxTimestamp()); return TriggerResult.CONTINUE; } @@ -73,8 +71,8 @@ public String toString() { /** * Creates a new trigger that fires once system time passes the end of the window. */ - public static ProcessingTimeTrigger create() { - return new ProcessingTimeTrigger<>(); + public static ProcessingTimeTrigger create() { + return new ProcessingTimeTrigger(); } } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala index 66767b1a0cd02..e8d5a1221bbb9 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala @@ -197,7 +197,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[EventTimeSessionWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -235,7 +235,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -268,7 +268,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -301,7 +301,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -340,7 +340,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -379,7 +379,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -419,7 +419,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -458,7 +458,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -497,7 +497,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -532,7 +532,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -569,7 +569,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -602,7 +602,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -635,7 +635,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -668,7 +668,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -701,7 +701,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -734,7 +734,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -771,7 +771,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -808,7 +808,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -841,7 +841,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -874,7 +874,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -916,7 +916,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -957,7 +957,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -998,7 +998,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1039,7 +1039,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1081,7 +1081,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1119,7 +1119,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1162,7 +1162,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1201,7 +1201,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1240,7 +1240,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1279,7 +1279,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1314,7 +1314,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1498,7 +1498,7 @@ class AllWindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1534,7 +1534,7 @@ class AllWindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1579,7 +1579,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1620,7 +1620,7 @@ class AllWindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala index 85b3d10ff4845..db0fb71c2f256 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala @@ -106,7 +106,7 @@ class TimeWindowTranslationTest extends AbstractTestBase { val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _, _]] - assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator1.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) } @@ -132,7 +132,7 @@ class TimeWindowTranslationTest extends AbstractTestBase { val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _, _]] - assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator1.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) } @@ -164,7 +164,7 @@ class TimeWindowTranslationTest extends AbstractTestBase { val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _, _]] - assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator1.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator1.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala index 95cf97fbca860..916884ff5ffc2 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala @@ -202,7 +202,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[EventTimeSessionWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -240,7 +240,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -274,7 +274,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -308,7 +308,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -349,7 +349,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -390,7 +390,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -432,7 +432,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -474,7 +474,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -515,7 +515,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -558,7 +558,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -595,7 +595,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]]) @@ -633,7 +633,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -667,7 +667,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -701,7 +701,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -735,7 +735,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -769,7 +769,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -803,7 +803,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -840,7 +840,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[AggregatingStateDescriptor[_, _, _]]) @@ -878,7 +878,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -912,7 +912,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -946,7 +946,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -990,7 +990,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1033,7 +1033,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1076,7 +1076,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1119,7 +1119,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1163,7 +1163,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1208,7 +1208,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1247,7 +1247,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[FoldingStateDescriptor[_, _]]) @@ -1292,7 +1292,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1333,7 +1333,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1374,7 +1374,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1415,7 +1415,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[ProcessingTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1451,7 +1451,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1643,7 +1643,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1680,7 +1680,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1716,7 +1716,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1751,7 +1751,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[WindowOperator[String, (String, Int), _, (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1787,7 +1787,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1828,7 +1828,7 @@ class WindowTranslationTest { .asInstanceOf[ EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[SlidingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1876,7 +1876,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) @@ -1919,7 +1919,7 @@ class WindowTranslationTest { val winOperator = operator .asInstanceOf[EvictingWindowOperator[String, (String, Int), (String, Int), _ <: Window]] - assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger[_]]) + assertTrue(winOperator.getTrigger.isInstanceOf[EventTimeTrigger]) assertTrue(winOperator.getEvictor.isInstanceOf[CountEvictor[_]]) assertTrue(winOperator.getWindowAssigner.isInstanceOf[TumblingEventTimeWindows]) assertTrue(winOperator.getStateDescriptor.isInstanceOf[ListStateDescriptor[_]]) From 963ac2e8553eea687f82ed3eba0b2e5fe92acfca Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Thu, 1 Feb 2018 13:07:14 +0000 Subject: [PATCH 5/7] [FLINK-8384] [streaming] Add dynamic gap convenience methods to static session window assigners --- .../windowing/assigners/EventTimeSessionWindows.java | 11 +++++++++++ .../assigners/ProcessingTimeSessionWindows.java | 11 +++++++++++ .../windowing/EventTimeSessionWindowsTest.java | 12 ++++++++++++ .../windowing/ProcessingTimeSessionWindowsTest.java | 12 ++++++++++++ 4 files changed, 46 insertions(+) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java index 249f219d36a1c..7e66478861e72 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java @@ -80,6 +80,17 @@ public static EventTimeSessionWindows withGap(Time size) { return new EventTimeSessionWindows(size.toMilliseconds()); } + /** + * Creates a new {@code SessionWindows} {@link WindowAssigner} that assigns + * elements to sessions based on the element timestamp. + * + * @param sessionWindowTimeGapExtractor The extractor to use to extract the time gap from the input elements + * @return The policy. + */ + public static DynamicEventTimeSessionWindows withDynamicGap(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { + return new DynamicEventTimeSessionWindows<>(sessionWindowTimeGapExtractor); + } + @Override public TypeSerializer getWindowSerializer(ExecutionConfig executionConfig) { return new TimeWindow.Serializer(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java index e5533109defae..a9e8fa57835e2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java @@ -81,6 +81,17 @@ public static ProcessingTimeSessionWindows withGap(Time size) { return new ProcessingTimeSessionWindows(size.toMilliseconds()); } + /** + * Creates a new {@code SessionWindows} {@link WindowAssigner} that assigns + * elements to sessions based on the element timestamp. + * + * @param sessionWindowTimeGapExtractor The extractor to use to extract the time gap from the input elements + * @return The policy. + */ + public static DynamicProcessingTimeSessionWindows withDynamicGap(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { + return new DynamicProcessingTimeSessionWindows<>(sessionWindowTimeGapExtractor); + } + @Override public TypeSerializer getWindowSerializer(ExecutionConfig executionConfig) { return new TimeWindow.Serializer(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java index 5c4c989338f50..7d96a1f60d4a5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java @@ -20,8 +20,10 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.DynamicEventTimeSessionWindows; import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner; +import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger; @@ -41,6 +43,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -186,4 +189,13 @@ public void testProperties() { assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig())); assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(EventTimeTrigger.class)); } + + @Test + public void testDynamicGapProperties() { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + DynamicEventTimeSessionWindows assigner = EventTimeSessionWindows.withDynamicGap(extractor); + + assertNotNull(assigner); + assertTrue(assigner.isEventTime()); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java index f49799cee99e3..5faca94df4fb9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java @@ -20,8 +20,10 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.DynamicProcessingTimeSessionWindows; import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner; import org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows; +import org.apache.flink.streaming.api.windowing.assigners.SessionWindowTimeGapExtractor; import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger; @@ -42,6 +44,7 @@ import static org.hamcrest.Matchers.instanceOf; 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.fail; import static org.mockito.Matchers.anyCollection; @@ -193,4 +196,13 @@ public void testProperties() { assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig())); assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(ProcessingTimeTrigger.class)); } + + @Test + public void testDynamicGapProperties() { + SessionWindowTimeGapExtractor extractor = mock(SessionWindowTimeGapExtractor.class); + DynamicProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withDynamicGap(extractor); + + assertNotNull(assigner); + assertFalse(assigner.isEventTime()); + } } From e26ff7a1bcdb314fa998ae2d6a3900b549559d52 Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Thu, 1 Feb 2018 13:26:47 +0000 Subject: [PATCH 6/7] [FLINK-8384] [streaming] Update documentation --- docs/dev/stream/operators/windows.md | 10 +++++----- .../assigners/SessionWindowTimeGapExtractor.java | 5 +++++ 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 836ce44ea1922..4deacce62b20d 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -303,7 +303,7 @@ input // event-time session windows with dynamic gap input .keyBy() - .window(DynamicEventTimeSessionWindows.withDynamicGap((element, timestamp, context) -> { + .window(EventTimeSessionWindows.withDynamicGap((element) -> { // determine and return session gap })) .(); @@ -317,7 +317,7 @@ input // processing-time session windows with dynamic gap input .keyBy() - .window(DynamicProcessingTimeSessionWindows.withDynamicGap((element, timestamp, context) -> { + .window(ProcessingTimeSessionWindows.withDynamicGap((element) -> { // determine and return session gap })) .(); @@ -337,8 +337,8 @@ input // event-time session windows with dynamic gap input .keyBy() - .window(DynamicEventTimeSessionWindows.withDynamicGap(new SessionWindowTimeGapExtractor[String] { - override def extract(element: String, ts: Long, ctx: WindowAssigner.WindowAssignerContext): Long = { + .window(EventTimeSessionWindows.withDynamicGap(new SessionWindowTimeGapExtractor[String] { + override def extract(element: String): Long = { // determine and return session gap } })) @@ -355,7 +355,7 @@ input input .keyBy() .window(DynamicProcessingTimeSessionWindows.withDynamicGap(new SessionWindowTimeGapExtractor[String] { - override def extract(element: String, ts: Long, ctx: WindowAssigner.WindowAssignerContext): Long = { + override def extract(element: String): Long = { // determine and return session gap } })) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java index 8b66cd4d486db..9f024a309247c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java @@ -26,5 +26,10 @@ * @param The type of elements that this {@code SessionWindowTimeGapExtractor} can extract session time gaps from. */ public interface SessionWindowTimeGapExtractor extends Serializable { + /** + * Extracts the session time gap. + * @param element The input element. + * @return The session time gap in milliseconds. + */ long extract(T element); } From 18fe9fa267b4f13ffd9eb0f3ebdf1f0e91b535ef Mon Sep 17 00:00:00 2001 From: Dyana Rose Date: Sun, 4 Feb 2018 15:45:43 +0000 Subject: [PATCH 7/7] [FLINK-8384] [streaming] Add PublicEvolving annotation --- .../windowing/assigners/DynamicEventTimeSessionWindows.java | 2 ++ .../assigners/DynamicProcessingTimeSessionWindows.java | 2 ++ .../api/windowing/assigners/EventTimeSessionWindows.java | 2 ++ .../api/windowing/assigners/ProcessingTimeSessionWindows.java | 2 ++ .../api/windowing/assigners/SessionWindowTimeGapExtractor.java | 3 +++ 5 files changed, 11 insertions(+) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java index 0f5c3ee852121..bba59ed91b084 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicEventTimeSessionWindows.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.windowing.assigners; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -42,6 +43,7 @@ * * @param The type of the input elements */ +@PublicEvolving public class DynamicEventTimeSessionWindows extends MergingWindowAssigner { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java index 05be06c9e81cf..073cc05958263 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.windowing.assigners; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -42,6 +43,7 @@ * * @param The type of the input elements */ +@PublicEvolving public class DynamicProcessingTimeSessionWindows extends MergingWindowAssigner { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java index 7e66478861e72..a33b28a8acc99 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.windowing.assigners; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -87,6 +88,7 @@ public static EventTimeSessionWindows withGap(Time size) { * @param sessionWindowTimeGapExtractor The extractor to use to extract the time gap from the input elements * @return The policy. */ + @PublicEvolving public static DynamicEventTimeSessionWindows withDynamicGap(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { return new DynamicEventTimeSessionWindows<>(sessionWindowTimeGapExtractor); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java index a9e8fa57835e2..8c5ab558f9772 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.windowing.assigners; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -88,6 +89,7 @@ public static ProcessingTimeSessionWindows withGap(Time size) { * @param sessionWindowTimeGapExtractor The extractor to use to extract the time gap from the input elements * @return The policy. */ + @PublicEvolving public static DynamicProcessingTimeSessionWindows withDynamicGap(SessionWindowTimeGapExtractor sessionWindowTimeGapExtractor) { return new DynamicProcessingTimeSessionWindows<>(sessionWindowTimeGapExtractor); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java index 9f024a309247c..5fa8a7940d4cc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SessionWindowTimeGapExtractor.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.windowing.assigners; +import org.apache.flink.annotation.PublicEvolving; + import java.io.Serializable; /** @@ -25,6 +27,7 @@ * * @param The type of elements that this {@code SessionWindowTimeGapExtractor} can extract session time gaps from. */ +@PublicEvolving public interface SessionWindowTimeGapExtractor extends Serializable { /** * Extracts the session time gap.