dawidwys commented on code in PR #23886: URL: https://github.com/apache/flink/pull/23886#discussion_r1540870351
########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java: ########## @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.AggregatePhaseStrategy; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.math.BigDecimal; +import java.util.function.Function; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecWindowAggregate}. */ +public class WindowAggregateTestPrograms { + + static final Row[] BEFORE_DATA = { + Row.of("2020-10-10 00:00:01", 1, 1d, 1f, new BigDecimal("1.11"), "Hi", "a"), + Row.of("2020-10-10 00:00:02", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:03", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:04", 5, 5d, 5f, new BigDecimal("5.55"), null, "a"), + Row.of("2020-10-10 00:00:07", 3, 3d, 3f, null, "Hello", "b"), + // out of order + Row.of("2020-10-10 00:00:06", 6, 6d, 6f, new BigDecimal("6.66"), "Hi", "b"), + Row.of("2020-10-10 00:00:08", 3, null, 3f, new BigDecimal("3.33"), "Comment#2", "a"), + // late event + Row.of("2020-10-10 00:00:04", 5, 5d, null, new BigDecimal("5.55"), "Hi", "a"), + Row.of("2020-10-10 00:00:16", 4, 4d, 4f, new BigDecimal("4.44"), "Hi", "b"), + Row.of("2020-10-10 00:00:32", 7, 7d, 7f, new BigDecimal("7.77"), null, null), + Row.of("2020-10-10 00:00:34", 1, 3d, 3f, new BigDecimal("3.33"), "Comment#3", "b") + }; + + static final Row[] AFTER_DATA = { + Row.of("2020-10-10 00:00:40", 10, 3d, 3f, new BigDecimal("4.44"), "Comment#4", "a"), + Row.of("2020-10-10 00:00:42", 11, 4d, 4f, new BigDecimal("5.44"), "Comment#5", "d"), + Row.of("2020-10-10 00:00:43", 12, 5d, 5f, new BigDecimal("6.44"), "Comment#6", "c"), + Row.of("2020-10-10 00:00:44", 13, 6d, 6f, new BigDecimal("7.44"), "Comment#7", "d") + }; + + static final Function<String, SourceTestStep.Builder> SOURCE_BUILDER = + str -> + SourceTestStep.newBuilder(str) + .addSchema( + "ts STRING", + "a_int INT", + "b_double DOUBLE", + "c_float FLOAT", + "d_bigdec DECIMAL(10, 2)", + "`comment` STRING", + "name STRING", + "`rowtime` AS TO_TIMESTAMP(`ts`)", + "`proctime` AS PROCTIME()", + "WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I,UA,UB,D") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA); + static final SourceTestStep SOURCE = SOURCE_BUILDER.apply("window_source_t").build(); + + static final SourceTestStep CDC_SOURCE = + SOURCE_BUILDER + .apply("cdc_window_source_t") + .addOption("changelog-mode", "I,UA,UB,D") + .build(); Review Comment: How are those two different? Are they even different? They both have exact same `changelog-mode`. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java: ########## @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.AggregatePhaseStrategy; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.math.BigDecimal; +import java.util.function.Function; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecWindowAggregate}. */ +public class WindowAggregateTestPrograms { + + static final Row[] BEFORE_DATA = { + Row.of("2020-10-10 00:00:01", 1, 1d, 1f, new BigDecimal("1.11"), "Hi", "a"), + Row.of("2020-10-10 00:00:02", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:03", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:04", 5, 5d, 5f, new BigDecimal("5.55"), null, "a"), + Row.of("2020-10-10 00:00:07", 3, 3d, 3f, null, "Hello", "b"), + // out of order + Row.of("2020-10-10 00:00:06", 6, 6d, 6f, new BigDecimal("6.66"), "Hi", "b"), + Row.of("2020-10-10 00:00:08", 3, null, 3f, new BigDecimal("3.33"), "Comment#2", "a"), + // late event + Row.of("2020-10-10 00:00:04", 5, 5d, null, new BigDecimal("5.55"), "Hi", "a"), + Row.of("2020-10-10 00:00:16", 4, 4d, 4f, new BigDecimal("4.44"), "Hi", "b"), + Row.of("2020-10-10 00:00:32", 7, 7d, 7f, new BigDecimal("7.77"), null, null), + Row.of("2020-10-10 00:00:34", 1, 3d, 3f, new BigDecimal("3.33"), "Comment#3", "b") + }; + + static final Row[] AFTER_DATA = { + Row.of("2020-10-10 00:00:40", 10, 3d, 3f, new BigDecimal("4.44"), "Comment#4", "a"), + Row.of("2020-10-10 00:00:42", 11, 4d, 4f, new BigDecimal("5.44"), "Comment#5", "d"), + Row.of("2020-10-10 00:00:43", 12, 5d, 5f, new BigDecimal("6.44"), "Comment#6", "c"), + Row.of("2020-10-10 00:00:44", 13, 6d, 6f, new BigDecimal("7.44"), "Comment#7", "d") + }; + + static final Function<String, SourceTestStep.Builder> SOURCE_BUILDER = + str -> + SourceTestStep.newBuilder(str) + .addSchema( + "ts STRING", + "a_int INT", + "b_double DOUBLE", + "c_float FLOAT", + "d_bigdec DECIMAL(10, 2)", + "`comment` STRING", + "name STRING", + "`rowtime` AS TO_TIMESTAMP(`ts`)", + "`proctime` AS PROCTIME()", + "WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I,UA,UB,D") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA); + static final SourceTestStep SOURCE = SOURCE_BUILDER.apply("window_source_t").build(); + + static final SourceTestStep CDC_SOURCE = + SOURCE_BUILDER + .apply("cdc_window_source_t") + .addOption("changelog-mode", "I,UA,UB,D") + .build(); + + static final String[] TUMBLE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-tumble-event-time", + "validates group by using tumbling window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS, + true); Review Comment: nit: It makes it way more readable if flags are enum. Just reading this line I don't know what the `true` here means. If it was `DISTINCT_SPLIT_ENABLED`, I would instantly know without looking elsewhere. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java: ########## @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.AggregatePhaseStrategy; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.math.BigDecimal; +import java.util.function.Function; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecWindowAggregate}. */ +public class WindowAggregateTestPrograms { + + static final Row[] BEFORE_DATA = { + Row.of("2020-10-10 00:00:01", 1, 1d, 1f, new BigDecimal("1.11"), "Hi", "a"), + Row.of("2020-10-10 00:00:02", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:03", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:04", 5, 5d, 5f, new BigDecimal("5.55"), null, "a"), + Row.of("2020-10-10 00:00:07", 3, 3d, 3f, null, "Hello", "b"), + // out of order + Row.of("2020-10-10 00:00:06", 6, 6d, 6f, new BigDecimal("6.66"), "Hi", "b"), + Row.of("2020-10-10 00:00:08", 3, null, 3f, new BigDecimal("3.33"), "Comment#2", "a"), + // late event + Row.of("2020-10-10 00:00:04", 5, 5d, null, new BigDecimal("5.55"), "Hi", "a"), + Row.of("2020-10-10 00:00:16", 4, 4d, 4f, new BigDecimal("4.44"), "Hi", "b"), + Row.of("2020-10-10 00:00:32", 7, 7d, 7f, new BigDecimal("7.77"), null, null), + Row.of("2020-10-10 00:00:34", 1, 3d, 3f, new BigDecimal("3.33"), "Comment#3", "b") + }; + + static final Row[] AFTER_DATA = { + Row.of("2020-10-10 00:00:40", 10, 3d, 3f, new BigDecimal("4.44"), "Comment#4", "a"), + Row.of("2020-10-10 00:00:42", 11, 4d, 4f, new BigDecimal("5.44"), "Comment#5", "d"), + Row.of("2020-10-10 00:00:43", 12, 5d, 5f, new BigDecimal("6.44"), "Comment#6", "c"), + Row.of("2020-10-10 00:00:44", 13, 6d, 6f, new BigDecimal("7.44"), "Comment#7", "d") + }; + + static final Function<String, SourceTestStep.Builder> SOURCE_BUILDER = + str -> + SourceTestStep.newBuilder(str) + .addSchema( + "ts STRING", + "a_int INT", + "b_double DOUBLE", + "c_float FLOAT", + "d_bigdec DECIMAL(10, 2)", + "`comment` STRING", + "name STRING", + "`rowtime` AS TO_TIMESTAMP(`ts`)", + "`proctime` AS PROCTIME()", + "WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I,UA,UB,D") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA); + static final SourceTestStep SOURCE = SOURCE_BUILDER.apply("window_source_t").build(); + + static final SourceTestStep CDC_SOURCE = + SOURCE_BUILDER + .apply("cdc_window_source_t") + .addOption("changelog-mode", "I,UA,UB,D") + .build(); + + static final String[] TUMBLE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-tumble-event-time", + "validates group by using tumbling window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:06, 2020-10-10T00:00:11, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:36, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:41, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:41, 2020-10-10T00:00:46, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-tumble-event-time-with-offset", + "validates group by using tumbling window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-with-offset", + "validates group by using tumbling window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] HOP_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-09T23:59:55, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:10, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:15, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:15, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:10, 2020-10-10T00:00:20, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:25, 1, 4, 1]" + }; + + public static final String[] HOP_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:25, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:25, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 7, 0]", + "+I[c, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 1, 12, 1]" + }; + + static final TableTestProgram HOP_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-hop-event-time", + "validates group by using a hop window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase", + "validates group by using a hop window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-distinct-split", + "validates group by using a hop window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-09T23:59:56, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:16, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:06, 2020-10-10T00:00:16, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:11, 2020-10-10T00:00:21, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:26, 1, 4, 1]" + }; + + public static final String[] HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:26, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:26, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 7, 0]", + "+I[c, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:41, 2020-10-10T00:00:51, 2, 24, 2]", + "+I[c, 2020-10-10T00:00:41, 2020-10-10T00:00:51, 1, 12, 1]" + }; + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-hop-event-time-with-offset", + "validates group by using a hop window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-with-offset", + "validates group by using a hop window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-with-offset-distinct-split", + "validates group by using a hop window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] CUMULATE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:10, 6, 18, 3]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:15, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:15, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:25, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:30, 1, 4, 1]" + }; + + public static final String[] CUMULATE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 1, 1]", + "+I[c, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 7, 0]" + }; + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-cumulate-event-time", + "validates group by using cumulate window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase", + "validates group by using cumulate window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase-distinct-split", + "validates group by using cumulate window with event time with two phase aggregation with distinct split", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:16, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:16, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:26, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:31, 1, 4, 1]" + }; + + public static final String[] CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 1, 1]", + "+I[c, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 10, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 7, 0]" + }; + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-cumulate-event-time-with-offset", + "validates group by using cumulate window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase-with-offset", + "validates group by using cumulate window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-cumulate-event-time-with-offset-distinct-split", + "validates group by using cumulate window with event time with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] SESSION_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 1, 1]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 1, 1]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 3, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 3, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 3, 5, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 3, 5, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 3, 1]", + "-U[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 3, 1]", + "+U[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 9, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 4, 10, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 5, 13, 3]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 5, 13, 3]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] SESSION_EVENT_TIME_AFTER_ROWS = { Review Comment: Does this need to be `public`? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java: ########## @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.AggregatePhaseStrategy; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.math.BigDecimal; +import java.util.function.Function; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecWindowAggregate}. */ +public class WindowAggregateTestPrograms { + + static final Row[] BEFORE_DATA = { Review Comment: Could we make all those fields `private`? If not, could you add a comment why they need to be visible? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java: ########## @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.AggregatePhaseStrategy; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.math.BigDecimal; +import java.util.function.Function; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecWindowAggregate}. */ +public class WindowAggregateTestPrograms { + + static final Row[] BEFORE_DATA = { + Row.of("2020-10-10 00:00:01", 1, 1d, 1f, new BigDecimal("1.11"), "Hi", "a"), + Row.of("2020-10-10 00:00:02", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:03", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:04", 5, 5d, 5f, new BigDecimal("5.55"), null, "a"), + Row.of("2020-10-10 00:00:07", 3, 3d, 3f, null, "Hello", "b"), + // out of order + Row.of("2020-10-10 00:00:06", 6, 6d, 6f, new BigDecimal("6.66"), "Hi", "b"), + Row.of("2020-10-10 00:00:08", 3, null, 3f, new BigDecimal("3.33"), "Comment#2", "a"), + // late event + Row.of("2020-10-10 00:00:04", 5, 5d, null, new BigDecimal("5.55"), "Hi", "a"), + Row.of("2020-10-10 00:00:16", 4, 4d, 4f, new BigDecimal("4.44"), "Hi", "b"), + Row.of("2020-10-10 00:00:32", 7, 7d, 7f, new BigDecimal("7.77"), null, null), + Row.of("2020-10-10 00:00:34", 1, 3d, 3f, new BigDecimal("3.33"), "Comment#3", "b") + }; + + static final Row[] AFTER_DATA = { + Row.of("2020-10-10 00:00:40", 10, 3d, 3f, new BigDecimal("4.44"), "Comment#4", "a"), + Row.of("2020-10-10 00:00:42", 11, 4d, 4f, new BigDecimal("5.44"), "Comment#5", "d"), + Row.of("2020-10-10 00:00:43", 12, 5d, 5f, new BigDecimal("6.44"), "Comment#6", "c"), + Row.of("2020-10-10 00:00:44", 13, 6d, 6f, new BigDecimal("7.44"), "Comment#7", "d") + }; + + static final Function<String, SourceTestStep.Builder> SOURCE_BUILDER = + str -> + SourceTestStep.newBuilder(str) + .addSchema( + "ts STRING", + "a_int INT", + "b_double DOUBLE", + "c_float FLOAT", + "d_bigdec DECIMAL(10, 2)", + "`comment` STRING", + "name STRING", + "`rowtime` AS TO_TIMESTAMP(`ts`)", + "`proctime` AS PROCTIME()", + "WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I,UA,UB,D") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA); + static final SourceTestStep SOURCE = SOURCE_BUILDER.apply("window_source_t").build(); + + static final SourceTestStep CDC_SOURCE = + SOURCE_BUILDER + .apply("cdc_window_source_t") + .addOption("changelog-mode", "I,UA,UB,D") + .build(); + + static final String[] TUMBLE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-tumble-event-time", + "validates group by using tumbling window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:06, 2020-10-10T00:00:11, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:36, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:41, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:41, 2020-10-10T00:00:46, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-tumble-event-time-with-offset", + "validates group by using tumbling window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-with-offset", + "validates group by using tumbling window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] HOP_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-09T23:59:55, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:10, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:15, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:15, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:10, 2020-10-10T00:00:20, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:25, 1, 4, 1]" + }; + + public static final String[] HOP_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:25, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:25, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 7, 0]", + "+I[c, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 1, 12, 1]" + }; + + static final TableTestProgram HOP_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-hop-event-time", + "validates group by using a hop window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase", + "validates group by using a hop window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-distinct-split", + "validates group by using a hop window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-09T23:59:56, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:16, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:06, 2020-10-10T00:00:16, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:11, 2020-10-10T00:00:21, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:26, 1, 4, 1]" + }; + + public static final String[] HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:26, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:26, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 7, 0]", + "+I[c, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:41, 2020-10-10T00:00:51, 2, 24, 2]", + "+I[c, 2020-10-10T00:00:41, 2020-10-10T00:00:51, 1, 12, 1]" + }; + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-hop-event-time-with-offset", + "validates group by using a hop window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-with-offset", + "validates group by using a hop window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-with-offset-distinct-split", + "validates group by using a hop window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] CUMULATE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:10, 6, 18, 3]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:15, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:15, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:25, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:30, 1, 4, 1]" + }; + + public static final String[] CUMULATE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 1, 1]", + "+I[c, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 7, 0]" + }; + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-cumulate-event-time", + "validates group by using cumulate window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase", + "validates group by using cumulate window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase-distinct-split", + "validates group by using cumulate window with event time with two phase aggregation with distinct split", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:16, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:16, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:26, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:31, 1, 4, 1]" + }; + + public static final String[] CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 1, 1]", + "+I[c, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 10, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 7, 0]" + }; + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-cumulate-event-time-with-offset", + "validates group by using cumulate window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase-with-offset", + "validates group by using cumulate window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-cumulate-event-time-with-offset-distinct-split", + "validates group by using cumulate window with event time with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] SESSION_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 1, 1]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 1, 1]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 3, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 3, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 3, 5, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 3, 5, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 3, 1]", + "-U[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 3, 1]", + "+U[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 9, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 4, 10, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 5, 13, 3]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 5, 13, 3]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] SESSION_EVENT_TIME_AFTER_ROWS = { + "+I[null, 2020-10-10T00:00:32, 2020-10-10T00:00:39, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:32, 2020-10-10T00:00:39, 1, 1, 1]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 11, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 12, 1]", + "-U[d, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 11, 1]", + "+U[d, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 2, 24, 2]" + }; + + static final TableTestProgram SESSION_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-session-event-time", + "validates group by using session window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "SESSION(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_BEFORE_ROWS, + SESSION_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-session-event-time-two-phase", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_BEFORE_ROWS, + SESSION_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-session-event-time-two-phase-distinct-split", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_BEFORE_ROWS, + SESSION_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS = { + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:12, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS = { + "+I[null, 2020-10-10T00:00:32, 2020-10-10T00:00:37, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:34, 2020-10-10T00:00:39, 1, 1, 1]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:43, 2020-10-10T00:00:48, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:42, 2020-10-10T00:00:49, 2, 24, 2]" + }; + + static final TableTestProgram SESSION_WINDOW_PARTITION_EVENT_TIME = + getTableTestProgram( + "window-aggregate-session-partition-event-time", + "validates group by using session window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "SESSION(TABLE window_source_t PARTITION BY name, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS, + SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_PARTITION_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-session-partition-event-time-two-phase", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE window_source_t PARTITION BY name, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS, + SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_PARTITION_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-session-partition-event-time-two-phase-distinct-split", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE cdc_window_source_t PARTITION BY name, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS, + SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS, + true); + + private static TableTestProgram getTableTestProgram( + final String id, + final String description, + final String aggPhaseStrategy, + final String windowSql, + final String[] beforeRows, + final String[] afterRows) { + return getTableTestProgram( + id, description, aggPhaseStrategy, windowSql, beforeRows, afterRows, false); + } + + private static TableTestProgram getTableTestProgram( + final String id, + final String description, + final String aggPhaseStrategy, + final String windowSql, + final String[] beforeRows, + final String[] afterRows, + final Boolean enableDistinctAggSplit) { Review Comment: Best would be to use an enum here, if not enum, at least `boolean`. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest.java: ########## @@ -1,528 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.plan.nodes.exec.stream; - -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.OptimizerConfigOptions; -import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.ConcatDistinctAggFunction; -import org.apache.flink.table.planner.utils.StreamTableTestUtil; -import org.apache.flink.table.planner.utils.TableTestBase; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -/** Test json serialization/deserialization for window aggregate. */ -class WindowAggregateJsonPlanTest extends TableTestBase { - - private StreamTableTestUtil util; - private TableEnvironment tEnv; - - @BeforeEach - void setup() { - util = streamTestUtil(TableConfig.getDefault()); - tEnv = util.getTableEnv(); - - String insertOnlyTableDdl = - "CREATE TABLE MyTable (\n" - + " a INT,\n" - + " b BIGINT,\n" - + " c VARCHAR,\n" - + " `rowtime` AS TO_TIMESTAMP(c),\n" - + " proctime as PROCTIME(),\n" - + " WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(insertOnlyTableDdl); - - String changelogTableDdl = - "CREATE TABLE MyCDCTable (\n" - + " a INT,\n" - + " b BIGINT,\n" - + " c VARCHAR,\n" - + " `rowtime` AS TO_TIMESTAMP(c),\n" - + " proctime as PROCTIME(),\n" - + " WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND\n" - + ") WITH (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D')\n"; - tEnv.executeSql(changelogTableDdl); - } - - @Test - void testEventTimeTumbleWindow() { - tEnv.createFunction("concat_distinct_agg", ConcatDistinctAggFunction.class); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_start TIMESTAMP(3),\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT,\n" - + " sum_a INT,\n" - + " distinct_cnt BIGINT,\n" - + " concat_distinct STRING\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_start,\n" - + " window_end,\n" - + " COUNT(*),\n" - + " SUM(a),\n" - + " COUNT(DISTINCT c),\n" - + " concat_distinct_agg(c)\n" - + "FROM TABLE(\n" - + " TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeTumbleWindowWithCDCSource() { - tEnv.createFunction("concat_distinct_agg", ConcatDistinctAggFunction.class); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_start TIMESTAMP(3),\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT,\n" - + " sum_a INT,\n" - + " distinct_cnt BIGINT,\n" - + " concat_distinct STRING\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_start,\n" - + " window_end,\n" - + " COUNT(*),\n" - + " SUM(a),\n" - + " COUNT(DISTINCT c),\n" - + " concat_distinct_agg(c)\n" - + "FROM TABLE(\n" - + " TUMBLE(TABLE MyCDCTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeTumbleWindowWithOffset() { - tEnv.createFunction("concat_distinct_agg", ConcatDistinctAggFunction.class); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_start TIMESTAMP(3),\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT,\n" - + " sum_a INT,\n" - + " distinct_cnt BIGINT,\n" - + " concat_distinct STRING\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_start,\n" - + " window_end,\n" - + " COUNT(*),\n" - + " SUM(a),\n" - + " COUNT(DISTINCT c),\n" - + " concat_distinct_agg(c)\n" - + "FROM TABLE(\n" - + " TUMBLE(\n" - + " TABLE MyTable,\n" - + " DESCRIPTOR(rowtime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '5' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testProcTimeTumbleWindow() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_end,\n" - + " COUNT(*)\n" - + "FROM TABLE(\n" - + " TUMBLE(TABLE MyTable, DESCRIPTOR(proctime), INTERVAL '15' MINUTE))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testProcTimeTumbleWindowWithCDCSource() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_end,\n" - + " COUNT(*)\n" - + "FROM TABLE(\n" - + " TUMBLE(TABLE MyCDCTable, DESCRIPTOR(proctime), INTERVAL '15' MINUTE))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeHopWindow() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " cnt BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " COUNT(c),\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " HOP(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeHopWindowWithCDCSource() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " cnt BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " COUNT(c),\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " HOP(TABLE MyCDCTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeHopWindowWithOffset() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " cnt BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " COUNT(c),\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " HOP(\n" - + " TABLE MyTable,\n" - + " DESCRIPTOR(rowtime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '10' SECOND,\n" - + " INTERVAL '5' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testProcTimeHopWindow() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " HOP(TABLE MyTable, DESCRIPTOR(proctime), INTERVAL '5' MINUTE, INTERVAL '10' MINUTE))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testProcTimeHopWindowWithCDCSource() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " HOP(TABLE MyCDCTable, DESCRIPTOR(proctime), INTERVAL '5' MINUTE, INTERVAL '10' MINUTE))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeCumulateWindow() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_end,\n" - + " COUNT(c),\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " CUMULATE(\n" - + " TABLE MyTable,\n" - + " DESCRIPTOR(rowtime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '15' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeCumulateWindowWithCDCSource() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_end,\n" - + " COUNT(c),\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " CUMULATE(\n" - + " TABLE MyCDCTable,\n" - + " DESCRIPTOR(rowtime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '15' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testEventTimeCumulateWindowWithOffset() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT,\n" - + " sum_a INT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " window_end,\n" - + " COUNT(c),\n" - + " SUM(a)\n" - + "FROM TABLE(\n" - + " CUMULATE(\n" - + " TABLE MyTable,\n" - + " DESCRIPTOR(rowtime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '15' SECOND,\n" - + " INTERVAL '15' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testProcTimeCumulateWindow() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " cnt BIGINT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " COUNT(c)\n" - + "FROM TABLE(\n" - + " CUMULATE(\n" - + " TABLE MyTable,\n" - + " DESCRIPTOR(proctime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '15' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testProcTimeCumulateWindowWithCDCSource() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " b BIGINT,\n" - + " cnt BIGINT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " b,\n" - + " COUNT(c)\n" - + "FROM TABLE(\n" - + " CUMULATE(\n" - + " TABLE MyCDCTable,\n" - + " DESCRIPTOR(proctime),\n" - + " INTERVAL '5' SECOND,\n" - + " INTERVAL '15' SECOND))\n" - + "GROUP BY b, window_start, window_end"); - } - - @Test - void testDistinctSplitEnabled() { - tEnv.getConfig() - .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a bigint,\n" - + " window_start timestamp(3),\n" - + " window_end timestamp(3),\n" - + " cnt_star bigint,\n" - + " sum_b bigint,\n" - + " cnt_distinct_c bigint\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - - util.verifyJsonPlan( - "insert into MySink select a, " - + " window_start, " - + " window_end, " - + " count(*), " - + " sum(b), " - + " count(distinct c) AS uv " - + "FROM TABLE (" - + " CUMULATE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '10' MINUTE, INTERVAL '1' HOUR)) " - + "GROUP BY a, window_start, window_end"); - } - - @Test - void testProcTimeSessionWindowWithoutPartitionKey() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " window_end TIMESTAMP(3),\n" - + " cnt BIGINT\n" - + ") WITH (\n" - + " 'connector' = 'values')\n"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "insert into MySink select\n" - + " window_end,\n" - + " COUNT(*)\n" - + "FROM TABLE(\n" - + " SESSION(TABLE MyTable, DESCRIPTOR(proctime), INTERVAL '15' MINUTE))\n" - + "GROUP BY window_start, window_end"); - } - - @Test - void testProcTimeSessionWindowWithPartitionKey() { Review Comment: Is the partitioning covered? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java: ########## @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.AggregatePhaseStrategy; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +import java.math.BigDecimal; +import java.util.function.Function; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecWindowAggregate}. */ +public class WindowAggregateTestPrograms { + + static final Row[] BEFORE_DATA = { + Row.of("2020-10-10 00:00:01", 1, 1d, 1f, new BigDecimal("1.11"), "Hi", "a"), + Row.of("2020-10-10 00:00:02", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:03", 2, 2d, 2f, new BigDecimal("2.22"), "Comment#1", "a"), + Row.of("2020-10-10 00:00:04", 5, 5d, 5f, new BigDecimal("5.55"), null, "a"), + Row.of("2020-10-10 00:00:07", 3, 3d, 3f, null, "Hello", "b"), + // out of order + Row.of("2020-10-10 00:00:06", 6, 6d, 6f, new BigDecimal("6.66"), "Hi", "b"), + Row.of("2020-10-10 00:00:08", 3, null, 3f, new BigDecimal("3.33"), "Comment#2", "a"), + // late event + Row.of("2020-10-10 00:00:04", 5, 5d, null, new BigDecimal("5.55"), "Hi", "a"), + Row.of("2020-10-10 00:00:16", 4, 4d, 4f, new BigDecimal("4.44"), "Hi", "b"), + Row.of("2020-10-10 00:00:32", 7, 7d, 7f, new BigDecimal("7.77"), null, null), + Row.of("2020-10-10 00:00:34", 1, 3d, 3f, new BigDecimal("3.33"), "Comment#3", "b") + }; + + static final Row[] AFTER_DATA = { + Row.of("2020-10-10 00:00:40", 10, 3d, 3f, new BigDecimal("4.44"), "Comment#4", "a"), + Row.of("2020-10-10 00:00:42", 11, 4d, 4f, new BigDecimal("5.44"), "Comment#5", "d"), + Row.of("2020-10-10 00:00:43", 12, 5d, 5f, new BigDecimal("6.44"), "Comment#6", "c"), + Row.of("2020-10-10 00:00:44", 13, 6d, 6f, new BigDecimal("7.44"), "Comment#7", "d") + }; + + static final Function<String, SourceTestStep.Builder> SOURCE_BUILDER = + str -> + SourceTestStep.newBuilder(str) + .addSchema( + "ts STRING", + "a_int INT", + "b_double DOUBLE", + "c_float FLOAT", + "d_bigdec DECIMAL(10, 2)", + "`comment` STRING", + "name STRING", + "`rowtime` AS TO_TIMESTAMP(`ts`)", + "`proctime` AS PROCTIME()", + "WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I,UA,UB,D") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA); + static final SourceTestStep SOURCE = SOURCE_BUILDER.apply("window_source_t").build(); + + static final SourceTestStep CDC_SOURCE = + SOURCE_BUILDER + .apply("cdc_window_source_t") + .addOption("changelog-mode", "I,UA,UB,D") + .build(); + + static final String[] TUMBLE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-tumble-event-time", + "validates group by using tumbling window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + TUMBLE_EVENT_TIME_BEFORE_ROWS, + TUMBLE_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:06, 2020-10-10T00:00:11, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:36, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:41, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:41, 2020-10-10T00:00:46, 2, 24, 2]" + }; + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-tumble-event-time-with-offset", + "validates group by using tumbling window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-with-offset", + "validates group by using tumbling window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split", + "validates group by using tumbling window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "TUMBLE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '1' SECOND)", + TUMBLE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + TUMBLE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] HOP_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-09T23:59:55, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:10, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:05, 2020-10-10T00:00:15, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:05, 2020-10-10T00:00:15, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:10, 2020-10-10T00:00:20, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:25, 1, 4, 1]" + }; + + public static final String[] HOP_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:25, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:25, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 7, 0]", + "+I[c, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:35, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:50, 1, 12, 1]" + }; + + static final TableTestProgram HOP_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-hop-event-time", + "validates group by using a hop window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase", + "validates group by using a hop window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-distinct-split", + "validates group by using a hop window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + HOP_EVENT_TIME_BEFORE_ROWS, + HOP_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-09T23:59:56, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:16, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:06, 2020-10-10T00:00:16, 1, 3, 1]", + "+I[b, 2020-10-10T00:00:11, 2020-10-10T00:00:21, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:26, 1, 4, 1]" + }; + + public static final String[] HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:26, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:26, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 7, 0]", + "+I[c, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:36, 2020-10-10T00:00:46, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:41, 2020-10-10T00:00:51, 2, 24, 2]", + "+I[c, 2020-10-10T00:00:41, 2020-10-10T00:00:51, 1, 12, 1]" + }; + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-hop-event-time-with-offset", + "validates group by using a hop window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-with-offset", + "validates group by using a hop window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-hop-event-time-two-phase-with-offset-distinct-split", + "validates group by using a hop window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "HOP(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND, INTERVAL '1' SECOND)", + HOP_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + HOP_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] CUMULATE_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:05, 4, 10, 2]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:10, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:10, 6, 18, 3]", + "+I[b, 2020-10-10T00:00, 2020-10-10T00:00:15, 2, 9, 2]", + "+I[a, 2020-10-10T00:00, 2020-10-10T00:00:15, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:20, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:25, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:15, 2020-10-10T00:00:30, 1, 4, 1]" + }; + + public static final String[] CUMULATE_EVENT_TIME_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:35, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:40, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 1, 1]", + "+I[c, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[null, 2020-10-10T00:00:30, 2020-10-10T00:00:45, 1, 7, 0]" + }; + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-cumulate-event-time", + "validates group by using cumulate window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase", + "validates group by using cumulate window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase-distinct-split", + "validates group by using cumulate window with event time with two phase aggregation with distinct split", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + CUMULATE_EVENT_TIME_BEFORE_ROWS, + CUMULATE_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:06, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:11, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:16, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:16, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:26, 1, 4, 1]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:31, 1, 4, 1]" + }; + + public static final String[] CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS = { + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:36, 1, 7, 0]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 10, 1]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 1, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:41, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 1, 1]", + "+I[c, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 2, 24, 2]", + "+I[a, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 10, 1]", + "+I[null, 2020-10-10T00:00:31, 2020-10-10T00:00:46, 1, 7, 0]" + }; + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-cumulate-event-time-with-offset", + "validates group by using cumulate window with event time with an offset", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET = + getTableTestProgram( + "window-aggregate-cumulate-event-time-two-phase-with-offset", + "validates group by using cumulate window with event time with two phase aggregation with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS); + + static final TableTestProgram CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-cumulate-event-time-with-offset-distinct-split", + "validates group by using cumulate window with event time with an offset", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "CUMULATE(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND, INTERVAL '1' SECOND)", + CUMULATE_EVENT_TIME_WITH_OFFSET_BEFORE_ROWS, + CUMULATE_EVENT_TIME_WITH_OFFSET_AFTER_ROWS, + true); + + static final String[] SESSION_EVENT_TIME_BEFORE_ROWS = { + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 1, 1]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 1, 1]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 3, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 3, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 3, 5, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 3, 5, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 4, 10, 2]", + "+I[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 3, 1]", + "-U[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 1, 3, 1]", + "+U[b, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 2, 9, 2]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 4, 10, 2]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 5, 13, 3]", + "-U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 5, 13, 3]", + "+U[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] SESSION_EVENT_TIME_AFTER_ROWS = { + "+I[null, 2020-10-10T00:00:32, 2020-10-10T00:00:39, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:32, 2020-10-10T00:00:39, 1, 1, 1]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 10, 1]", + "+I[d, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 11, 1]", + "+I[c, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 12, 1]", + "-U[d, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 1, 11, 1]", + "+U[d, 2020-10-10T00:00:40, 2020-10-10T00:00:49, 2, 24, 2]" + }; + + static final TableTestProgram SESSION_WINDOW_EVENT_TIME = + getTableTestProgram( + "window-aggregate-session-event-time", + "validates group by using session window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "SESSION(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_BEFORE_ROWS, + SESSION_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-session-event-time-two-phase", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_BEFORE_ROWS, + SESSION_EVENT_TIME_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-session-event-time-two-phase-distinct-split", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE window_source_t, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_BEFORE_ROWS, + SESSION_EVENT_TIME_AFTER_ROWS, + true); + + static final String[] SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS = { + "+I[b, 2020-10-10T00:00:06, 2020-10-10T00:00:12, 2, 9, 2]", + "+I[a, 2020-10-10T00:00:01, 2020-10-10T00:00:13, 6, 18, 3]", + "+I[b, 2020-10-10T00:00:16, 2020-10-10T00:00:21, 1, 4, 1]" + }; + + public static final String[] SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS = { + "+I[null, 2020-10-10T00:00:32, 2020-10-10T00:00:37, 1, 7, 0]", + "+I[b, 2020-10-10T00:00:34, 2020-10-10T00:00:39, 1, 1, 1]", + "+I[a, 2020-10-10T00:00:40, 2020-10-10T00:00:45, 1, 10, 1]", + "+I[c, 2020-10-10T00:00:43, 2020-10-10T00:00:48, 1, 12, 1]", + "+I[d, 2020-10-10T00:00:42, 2020-10-10T00:00:49, 2, 24, 2]" + }; + + static final TableTestProgram SESSION_WINDOW_PARTITION_EVENT_TIME = + getTableTestProgram( + "window-aggregate-session-partition-event-time", + "validates group by using session window with event time", + AggregatePhaseStrategy.ONE_PHASE.toString(), + "SESSION(TABLE window_source_t PARTITION BY name, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS, + SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_PARTITION_EVENT_TIME_TWO_PHASE = + getTableTestProgram( + "window-aggregate-session-partition-event-time-two-phase", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE window_source_t PARTITION BY name, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + SESSION_EVENT_TIME_PARTITIONED_BEFORE_ROWS, + SESSION_EVENT_TIME_PARTITIONED_AFTER_ROWS); + + static final TableTestProgram SESSION_WINDOW_PARTITION_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT = + getTableTestProgram( + "window-aggregate-session-partition-event-time-two-phase-distinct-split", + "validates group by using session window with event time with two phase aggregation", + AggregatePhaseStrategy.TWO_PHASE.toString(), + "SESSION(TABLE cdc_window_source_t PARTITION BY name, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", Review Comment: Why does this one test use a different source? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
