Repository: flink Updated Branches: refs/heads/master b0753f193 -> 2c20b39ff
[FLINK-4866] [streaming] Make Trigger.clear() Abstract to Enforce Implementation Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2c20b39f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2c20b39f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2c20b39f Branch: refs/heads/master Commit: 2c20b39ff03b19363a14024836f99098b52611f5 Parents: b0753f1 Author: Jark Wu <[email protected]> Authored: Fri Oct 21 13:46:45 2016 +0800 Committer: Aljoscha Krettek <[email protected]> Committed: Mon Oct 24 09:11:46 2016 +0200 ---------------------------------------------------------------------- .../apache/flink/streaming/api/windowing/triggers/Trigger.java | 4 +--- .../runtime/operators/windowing/AllWindowTranslationTest.java | 3 +++ .../runtime/operators/windowing/WindowTranslationTest.java | 3 +++ 3 files changed, 7 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/2c20b39f/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java index ff80639..3f68e78 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java @@ -113,10 +113,8 @@ public abstract class Trigger<T, W extends Window> implements Serializable { * when a window is purged. Timers set using {@link TriggerContext#registerEventTimeTimer(long)} * and {@link TriggerContext#registerProcessingTimeTimer(long)} should be deleted here as * well as state acquired using {@link TriggerContext#getPartitionedState(StateDescriptor)}. - * - * <p>By default, this method does nothing. */ - public void clear(W window, TriggerContext ctx) throws Exception {} + public abstract void clear(W window, TriggerContext ctx) throws Exception; // ------------------------------------------------------------------------ http://git-wip-us.apache.org/repos/asf/flink/blob/2c20b39f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java index 7ac9e13..72b0850 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java @@ -308,6 +308,9 @@ public class AllWindowTranslationTest { public boolean canMerge() { return false; } + + @Override + public void clear(TimeWindow window, TriggerContext ctx) throws Exception {} }); } catch (UnsupportedOperationException e) { // expected http://git-wip-us.apache.org/repos/asf/flink/blob/2c20b39f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java index 2707108..66de849 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java @@ -318,6 +318,9 @@ public class WindowTranslationTest { public boolean canMerge() { return false; } + + @Override + public void clear(TimeWindow window, TriggerContext ctx) throws Exception {} }); } catch (UnsupportedOperationException e) { // expected
