Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id EEF60200CAF for ; Wed, 7 Jun 2017 18:30:01 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id ED7AA160BD0; Wed, 7 Jun 2017 16:30:01 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 25B67160BED for ; Wed, 7 Jun 2017 18:29:59 +0200 (CEST) Received: (qmail 51002 invoked by uid 500); 7 Jun 2017 16:29:59 -0000 Mailing-List: contact commits-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list commits@flink.apache.org Received: (qmail 50906 invoked by uid 99); 7 Jun 2017 16:29:59 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 07 Jun 2017 16:29:59 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1B5E6DFE22; Wed, 7 Jun 2017 16:29:59 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tzulitai@apache.org To: commits@flink.apache.org Date: Wed, 07 Jun 2017 16:30:07 -0000 Message-Id: In-Reply-To: <4f8737a1009446dda2a424349f97f63e@git.apache.org> References: <4f8737a1009446dda2a424349f97f63e@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [10/12] flink git commit: [FLINK-6830] [DataStream] Port window operator migration tests for Flink 1.3 archived-at: Wed, 07 Jun 2017 16:30:02 -0000 http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java new file mode 100644 index 0000000..9f19064 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java @@ -0,0 +1,1087 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ReducingStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.TypeInfoParser; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.LegacyWindowOperatorType; +import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.WindowFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger; +import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger; +import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.streaming.api.windowing.windows.Window; +import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction; +import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueWindowFunction; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OperatorSnapshotUtil; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.streaming.util.migration.MigrationTestUtil; +import org.apache.flink.streaming.util.migration.MigrationVersion; +import org.apache.flink.util.Collector; + +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.fail; + +/** + * Tests for checking whether {@link WindowOperator} can restore from snapshots that were done + * using previous Flink versions' {@link WindowOperator}. + * + *

This also checks whether {@link WindowOperator} can restore from a checkpoint of the + * aligned processing-time windows operator of previous Flink versions. + * + *

For regenerating the binary snapshot file you have to run the {@code write*()} method on + * the corresponding Flink release-* branch. + */ +@RunWith(Parameterized.class) +public class WindowOperatorMigrationTest { + + @Parameterized.Parameters(name = "Migration Savepoint: {0}") + public static Collection parameters () { + return Arrays.asList(MigrationVersion.v1_1, MigrationVersion.v1_2, MigrationVersion.v1_3); + } + + /** + * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3) + * TODO and remove all @Ignore annotations on write*Snapshot() methods to generate savepoints + */ + private final MigrationVersion flinkGenerateSavepointVersion = null; + + private final MigrationVersion testMigrateVersion; + + public WindowOperatorMigrationTest(MigrationVersion testMigrateVersion) { + this.testMigrateVersion = testMigrateVersion; + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeSessionWindowsWithCountTriggerSnapshot() throws Exception { + final int sessionSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple3, TimeWindow> operator = new WindowOperator<>( + EventTimeSessionWindows.withGap(Time.seconds(sessionSize)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new SessionWindowFunction()), + PurgingTrigger.of(CountTrigger.of(4)), + 0, + null /* late data output tag */); + + OneInputStreamOperatorTestHarness, Tuple3> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + // add elements out-of-order + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000)); + + // do snapshot and save to file + OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L); + + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink" + flinkGenerateSavepointVersion + "-snapshot"); + + testHarness.close(); + } + + @Test + public void testRestoreSessionWindowsWithCountTrigger() throws Exception { + + final int sessionSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple3, TimeWindow> operator = new WindowOperator<>( + EventTimeSessionWindows.withGap(Time.seconds(sessionSize)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new SessionWindowFunction()), + PurgingTrigger.of(CountTrigger.of(4)), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple3> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-session-with-stateful-trigger-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); + + // add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500)); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); + + testHarness.close(); + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeSessionWindowsWithCountTriggerInMintConditionSnapshot() throws Exception { + + final int sessionSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple3, TimeWindow> operator = new WindowOperator<>( + EventTimeSessionWindows.withGap(Time.seconds(sessionSize)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new SessionWindowFunction()), + PurgingTrigger.of(CountTrigger.of(4)), + 0, + null /* late data output tag */); + + OneInputStreamOperatorTestHarness, Tuple3> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + // do snapshot and save to file + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink" + flinkGenerateSavepointVersion + "-snapshot"); + + testHarness.close(); + } + + /** + * This checks that we can restore from a virgin {@code WindowOperator} that has never seen + * any elements. + */ + @Test + public void testRestoreSessionWindowsWithCountTriggerInMintCondition() throws Exception { + + final int sessionSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple3, TimeWindow> operator = new WindowOperator<>( + EventTimeSessionWindows.withGap(Time.seconds(sessionSize)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new SessionWindowFunction()), + PurgingTrigger.of(CountTrigger.of(4)), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple3> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-session-with-stateful-trigger-mint-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + // add elements out-of-order + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000)); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-10", 0L, 6500L), 6499)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); + + // add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500)); + + expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); + + testHarness.close(); + } + + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeReducingEventTimeWindowsSnapshot() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", + new SumReducer(), + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingEventTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction>()), + EventTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + // add elements out-of-order + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); + + testHarness.processWatermark(new Watermark(999)); + expectedOutput.add(new Watermark(999)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + testHarness.processWatermark(new Watermark(1999)); + expectedOutput.add(new Watermark(1999)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + // do snapshot and save to file + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-reduce-event-time-flink" + flinkGenerateSavepointVersion + "-snapshot"); + + testHarness.close(); + } + + @Test + public void testRestoreReducingEventTimeWindows() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", + new SumReducer(), + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingEventTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction>()), + EventTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-reduce-event-time-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.processWatermark(new Watermark(2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999)); + expectedOutput.add(new Watermark(2999)); + + testHarness.processWatermark(new Watermark(3999)); + expectedOutput.add(new Watermark(3999)); + + testHarness.processWatermark(new Watermark(4999)); + expectedOutput.add(new Watermark(4999)); + + testHarness.processWatermark(new Watermark(5999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999)); + expectedOutput.add(new Watermark(5999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + testHarness.close(); + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeApplyEventTimeWindowsSnapshot() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingEventTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new RichSumReducer()), + EventTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + // add elements out-of-order + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000)); + + testHarness.processWatermark(new Watermark(999)); + expectedOutput.add(new Watermark(999)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + testHarness.processWatermark(new Watermark(1999)); + expectedOutput.add(new Watermark(1999)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + // do snapshot and save to file + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-apply-event-time-flink" + flinkGenerateSavepointVersion + "-snapshot"); + + testHarness.close(); + } + + @Test + public void testRestoreApplyEventTimeWindows() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingEventTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new RichSumReducer()), + EventTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-apply-event-time-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.processWatermark(new Watermark(2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999)); + expectedOutput.add(new Watermark(2999)); + + testHarness.processWatermark(new Watermark(3999)); + expectedOutput.add(new Watermark(3999)); + + testHarness.processWatermark(new Watermark(4999)); + expectedOutput.add(new Watermark(4999)); + + testHarness.processWatermark(new Watermark(5999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999)); + expectedOutput.add(new Watermark(5999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + testHarness.close(); + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeReducingProcessingTimeWindowsSnapshot() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", + new SumReducer(), + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingProcessingTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction>()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(10); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1))); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1))); + + testHarness.setProcessingTime(3010); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1))); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1))); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + // do snapshot and save to file + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-reduce-processing-time-flink" + flinkGenerateSavepointVersion + "-snapshot"); + + testHarness.close(); + + } + + @Test + public void testRestoreReducingProcessingTimeWindows() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", + new SumReducer(), + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingProcessingTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction>()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-reduce-processing-time-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.setProcessingTime(3020); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3))); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3))); + + testHarness.setProcessingTime(6000); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + testHarness.close(); + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeApplyProcessingTimeWindowsSnapshot() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingProcessingTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new RichSumReducer()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(10); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1))); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1))); + + testHarness.setProcessingTime(3010); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1))); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1))); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + // do snapshot and save to file + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-apply-processing-time-flink" + flinkGenerateSavepointVersion + "-snapshot"); + + testHarness.close(); + } + + @Test + public void testRestoreApplyProcessingTimeWindows() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ListStateDescriptor> stateDesc = new ListStateDescriptor<>("window-contents", + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Iterable>, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingProcessingTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalIterableWindowFunction<>(new RichSumReducer()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-apply-processing-time-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.setProcessingTime(3020); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3))); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3))); + + testHarness.setProcessingTime(6000); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + testHarness.close(); + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeAggregatingAlignedProcessingTimeWindowsSnapshot() throws Exception { + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + AggregatingProcessingTimeWindowOperator> operator = + new AggregatingProcessingTimeWindowOperator<>( + new ReduceFunction>() { + private static final long serialVersionUID = -8913160567151867987L; + + @Override + public Tuple2 reduce(Tuple2 value1, Tuple2 value2) throws Exception { + return new Tuple2<>(value1.f0, value1.f1 + value2.f1); + } + }, + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + inputType.createSerializer(new ExecutionConfig()), + 3000, + 3000); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.open(); + + testHarness.setProcessingTime(3); + + // timestamp is ignored in processing time + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + + // do a snapshot, close and restore again + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-aggr-aligned-flink" + flinkGenerateSavepointVersion + "-snapshot"); + testHarness.close(); + } + + @Test + public void testRestoreAggregatingAlignedProcessingTimeWindows() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", + new SumReducer(), + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingProcessingTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction>()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */, + LegacyWindowOperatorType.FAST_AGGREGATING); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-aggr-aligned-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.setProcessingTime(5000); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + + testHarness.setProcessingTime(7000); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + testHarness.close(); + } + + /** + * Manually run this to write binary snapshot data. + */ + @Ignore + @Test + public void writeAlignedProcessingTimeWindowsSnapshot() throws Exception { + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + AccumulatingProcessingTimeWindowOperator, Tuple2> operator = + new AccumulatingProcessingTimeWindowOperator<>( + new InternalIterableWindowFunction<>(new WindowFunction, Tuple2, String, TimeWindow>() { + + private static final long serialVersionUID = 6551516443265733803L; + + @Override + public void apply(String s, TimeWindow window, Iterable> input, Collector> out) throws Exception { + int sum = 0; + for (Tuple2 anInput : input) { + sum += anInput.f1; + } + out.collect(new Tuple2<>(s, sum)); + } + }), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + inputType.createSerializer(new ExecutionConfig()), + 3000, + 3000); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + testHarness.open(); + + testHarness.setProcessingTime(3); + + // timestamp is ignored in processing time + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000)); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + + // do a snapshot, close and restore again + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + OperatorSnapshotUtil.writeStateHandle( + snapshot, + "src/test/resources/win-op-migration-test-accum-aligned-flink" + flinkGenerateSavepointVersion + "-snapshot"); + testHarness.close(); + } + + @Test + public void testRestoreAccumulatingAlignedProcessingTimeWindows() throws Exception { + final int windowSize = 3; + + TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); + + ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", + new SumReducer(), + inputType.createSerializer(new ExecutionConfig())); + + WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( + TumblingProcessingTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), + new TimeWindow.Serializer(), + new TupleKeySelector(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), + stateDesc, + new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction>()), + ProcessingTimeTrigger.create(), + 0, + null /* late data output tag */, + LegacyWindowOperatorType.FAST_ACCUMULATING); + + OneInputStreamOperatorTestHarness, Tuple2> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.setup(); + + MigrationTestUtil.restoreFromSnapshot( + testHarness, + OperatorSnapshotUtil.getResourceFilename( + "win-op-migration-test-accum-aligned-flink" + testMigrateVersion + "-snapshot"), + testMigrateVersion); + + testHarness.open(); + + testHarness.setProcessingTime(5000); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999)); + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000)); + + testHarness.setProcessingTime(7000); + + expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator()); + + testHarness.close(); + } + + private static class TupleKeySelector implements KeySelector, String> { + private static final long serialVersionUID = 1L; + + @Override + public String getKey(Tuple2 value) throws Exception { + return value.f0; + } + } + + @SuppressWarnings("unchecked") + private static class Tuple2ResultSortComparator implements Comparator { + @Override + public int compare(Object o1, Object o2) { + if (o1 instanceof Watermark || o2 instanceof Watermark) { + return 0; + } else { + StreamRecord> sr0 = (StreamRecord>) o1; + StreamRecord> sr1 = (StreamRecord>) o2; + if (sr0.getTimestamp() != sr1.getTimestamp()) { + return (int) (sr0.getTimestamp() - sr1.getTimestamp()); + } + int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0); + if (comparison != 0) { + return comparison; + } else { + return sr0.getValue().f1 - sr1.getValue().f1; + } + } + } + } + + @SuppressWarnings("unchecked") + private static class Tuple3ResultSortComparator implements Comparator { + @Override + public int compare(Object o1, Object o2) { + if (o1 instanceof Watermark || o2 instanceof Watermark) { + return 0; + } else { + StreamRecord> sr0 = (StreamRecord>) o1; + StreamRecord> sr1 = (StreamRecord>) o2; + if (sr0.getTimestamp() != sr1.getTimestamp()) { + return (int) (sr0.getTimestamp() - sr1.getTimestamp()); + } + int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0); + if (comparison != 0) { + return comparison; + } else { + comparison = (int) (sr0.getValue().f1 - sr1.getValue().f1); + if (comparison != 0) { + return comparison; + } + return (int) (sr0.getValue().f1 - sr1.getValue().f1); + } + } + } + } + + private static class SumReducer implements ReduceFunction> { + private static final long serialVersionUID = 1L; + @Override + public Tuple2 reduce(Tuple2 value1, + Tuple2 value2) throws Exception { + return new Tuple2<>(value2.f0, value1.f1 + value2.f1); + } + } + + private static class RichSumReducer extends RichWindowFunction, Tuple2, String, W> { + private static final long serialVersionUID = 1L; + + private boolean openCalled = false; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + openCalled = true; + } + + @Override + public void close() throws Exception { + super.close(); + } + + @Override + public void apply(String key, + W window, + Iterable> input, + Collector> out) throws Exception { + + if (!openCalled) { + fail("Open was not called"); + } + int sum = 0; + + for (Tuple2 t: input) { + sum += t.f1; + } + out.collect(new Tuple2<>(key, sum)); + + } + + } + + private static class SessionWindowFunction implements WindowFunction, Tuple3, String, TimeWindow> { + private static final long serialVersionUID = 1L; + + @Override + public void apply(String key, + TimeWindow window, + Iterable> values, + Collector> out) throws Exception { + int sum = 0; + for (Tuple2 i: values) { + sum += i.f1; + } + String resultString = key + "-" + sum; + out.collect(new Tuple3<>(resultString, window.getStart(), window.getEnd())); + } + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot new file mode 100644 index 0000000..f85adf7 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot new file mode 100644 index 0000000..7ed2ab9 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot new file mode 100644 index 0000000..be4b306 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot new file mode 100644 index 0000000..a642a3a Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot new file mode 100644 index 0000000..a9f0075 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot new file mode 100644 index 0000000..7610073 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot new file mode 100644 index 0000000..8ad2d39 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot differ http://git-wip-us.apache.org/repos/asf/flink/blob/9f6407dc/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot new file mode 100644 index 0000000..7337bae Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot differ