From 0dc2a0f74cf8d8ac10a54fa6c4f0c1f32dc2e273 Mon Sep 17 00:00:00 2001 From: Jim Hughes Date: Fri, 22 Mar 2024 18:18:13 -0400 Subject: [PATCH] [FLINK-33676] Implement RestoreTests for WindowAggregate This closes #23886 --- .../WindowAggregateEventTimeRestoreTest.java | 65 ++ .../stream/WindowAggregateTestPrograms.java | 538 +++++++++++ ...EventTimeSessionWindowWithPartitionKey.out | 373 ------- ...ntTimeSessionWindowWithoutPartitionKey.out | 355 ------- ...estProcTimeCumulateWindowWithCDCSource.out | 474 --------- .../testProcTimeHopWindow.out | 473 --------- .../testProcTimeHopWindowWithCDCSource.out | 474 --------- ...tProcTimeSessionWindowWithPartitionKey.out | 460 --------- ...ocTimeSessionWindowWithoutPartitionKey.out | 432 --------- .../testProcTimeTumbleWindow.out | 459 --------- .../testProcTimeTumbleWindowWithCDCSource.out | 460 --------- ...e-event-time-two-phase-distinct-split.json | 908 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 29761 bytes ...ate-event-time-two-phase-with-offset.json} | 315 +++--- .../savepoint/_metadata | Bin 0 -> 23185 bytes ...gregate-cumulate-event-time-two-phase.json | 647 +++++++++++++ .../savepoint/_metadata | Bin 0 -> 23181 bytes ...event-time-with-offset-distinct-split.json | 912 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 29761 bytes ...egate-cumulate-event-time-with-offset.json | 499 ++++++++++ .../savepoint/_metadata | Bin 0 -> 23134 bytes .../window-aggregate-cumulate-event-time.json | 498 ++++++++++ .../savepoint/_metadata | Bin 0 -> 23134 bytes ...p-event-time-two-phase-distinct-split.json | 908 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 30037 bytes ...-two-phase-with-offset-distinct-split.json | 912 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 30031 bytes ...-hop-event-time-two-phase-with-offset.json | 649 +++++++++++++ .../savepoint/_metadata | Bin 0 -> 23049 bytes ...ow-aggregate-hop-event-time-two-phase.json | 647 +++++++++++++ .../savepoint/_metadata | Bin 0 -> 23045 bytes ...aggregate-hop-event-time-with-offset.json} | 308 +++--- .../savepoint/_metadata | Bin 0 -> 22998 bytes .../window-aggregate-hop-event-time.json} | 305 +++--- .../savepoint/_metadata | Bin 0 -> 22994 bytes ...-event-time-two-phase-distinct-split.json} | 363 ++++--- .../savepoint/_metadata | Bin 0 -> 31910 bytes ...ggregate-session-event-time-two-phase.json | 491 ++++++++++ .../savepoint/_metadata | Bin 0 -> 31476 bytes .../window-aggregate-session-event-time.json | 491 ++++++++++ .../savepoint/_metadata | Bin 0 -> 31472 bytes ...-event-time-two-phase-distinct-split.json} | 266 +++-- .../savepoint/_metadata | Bin 0 -> 23641 bytes ...ession-partition-event-time-two-phase.json | 499 ++++++++++ .../savepoint/_metadata | Bin 0 -> 23215 bytes ...gregate-session-partition-event-time.json} | 267 +++-- .../savepoint/_metadata | Bin 0 -> 23219 bytes ...-event-time-two-phase-distinct-split.json} | 337 +++---- .../savepoint/_metadata | Bin 0 -> 29223 bytes ...-two-phase-with-offset-distinct-split.json | 908 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 29223 bytes ...ble-event-time-two-phase-with-offset.json} | 300 ++---- .../savepoint/_metadata | Bin 0 -> 22637 bytes ...ggregate-tumble-event-time-two-phase.json} | 296 ++---- .../savepoint/_metadata | Bin 0 -> 22637 bytes ...gregate-tumble-event-time-with-offset.json | 498 ++++++++++ .../savepoint/_metadata | Bin 0 -> 22590 bytes .../window-aggregate-tumble-event-time.json} | 265 +++-- .../savepoint/_metadata | Bin 0 -> 22586 bytes .../window-aggregate-tumble-proc-time.json} | 271 +++--- .../savepoint/_metadata | Bin 0 -> 18217 bytes 61 files changed, 11500 insertions(+), 5823 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateEventTimeRestoreTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithPartitionKey.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithoutPartitionKey.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindowWithCDCSource.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindowWithCDCSource.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithPartitionKey.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithoutPartitionKey.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindowWithCDCSource.out create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/plan/window-aggregate-cumulate-event-time-two-phase-distinct-split.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithCDCSource.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/plan/window-aggregate-cumulate-event-time-two-phase-with-offset.json} (64%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/plan/window-aggregate-cumulate-event-time-two-phase.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/plan/window-aggregate-cumulate-event-time-with-offset-distinct-split.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/plan/window-aggregate-cumulate-event-time-with-offset.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/plan/window-aggregate-cumulate-event-time.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-distinct-split/plan/window-aggregate-hop-event-time-two-phase-distinct-split.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-distinct-split/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset-distinct-split/plan/window-aggregate-hop-event-time-two-phase-with-offset-distinct-split.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset-distinct-split/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/plan/window-aggregate-hop-event-time-two-phase-with-offset.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase/plan/window-aggregate-hop-event-time-two-phase.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/plan/window-aggregate-hop-event-time-with-offset.json} (67%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/plan/window-aggregate-hop-event-time.json} (67%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithCDCSource.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/plan/window-aggregate-session-event-time-two-phase-distinct-split.json} (62%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/plan/window-aggregate-session-event-time-two-phase.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/plan/window-aggregate-session-event-time.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithCDCSource.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/plan/window-aggregate-session-partition-event-time-two-phase-distinct-split.json} (70%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase/plan/window-aggregate-session-partition-event-time-two-phase.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time/plan/window-aggregate-session-partition-event-time.json} (70%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-distinct-split/plan/window-aggregate-tumble-event-time-two-phase-distinct-split.json} (72%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-distinct-split/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/plan/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/plan/window-aggregate-tumble-event-time-two-phase-with-offset.json} (65%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/plan/window-aggregate-tumble-event-time-two-phase.json} (65%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-with-offset/plan/window-aggregate-tumble-event-time-with-offset.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-with-offset/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/plan/window-aggregate-tumble-event-time.json} (70%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/savepoint/_metadata rename flink-table/flink-table-planner/src/test/resources/{org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out => restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/plan/window-aggregate-tumble-proc-time.json} (66%) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/savepoint/_metadata diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateEventTimeRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateEventTimeRestoreTest.java new file mode 100644 index 00000000000..5ef4c962245 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateEventTimeRestoreTest.java @@ -0,0 +1,65 @@ +/* + * 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.planner.plan.nodes.exec.testutils.RestoreTestBase; +import org.apache.flink.table.test.program.TableTestProgram; + +import java.util.Arrays; +import java.util.List; + +/** Restore tests for {@link StreamExecWindowAggregate}. */ +public class WindowAggregateEventTimeRestoreTest extends RestoreTestBase { + + public WindowAggregateEventTimeRestoreTest() { + super(StreamExecWindowAggregate.class); + } + + @Override + public List programs() { + return Arrays.asList( + WindowAggregateTestPrograms.TUMBLE_WINDOW_EVENT_TIME, + WindowAggregateTestPrograms.TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE, + WindowAggregateTestPrograms.TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT, + WindowAggregateTestPrograms.TUMBLE_WINDOW_EVENT_TIME_WITH_OFFSET, + WindowAggregateTestPrograms.TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET, + WindowAggregateTestPrograms + .TUMBLE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT, + WindowAggregateTestPrograms.HOP_WINDOW_EVENT_TIME, + WindowAggregateTestPrograms.HOP_WINDOW_EVENT_TIME_TWO_PHASE, + WindowAggregateTestPrograms.HOP_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT, + WindowAggregateTestPrograms.HOP_WINDOW_EVENT_TIME_WITH_OFFSET, + WindowAggregateTestPrograms.HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET, + WindowAggregateTestPrograms + .HOP_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET_DISTINCT_SPLIT, + WindowAggregateTestPrograms.CUMULATE_WINDOW_EVENT_TIME, + WindowAggregateTestPrograms.CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE, + WindowAggregateTestPrograms.CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT, + WindowAggregateTestPrograms.CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET, + WindowAggregateTestPrograms.CUMULATE_WINDOW_EVENT_TIME_TWO_PHASE_WITH_OFFSET, + WindowAggregateTestPrograms.CUMULATE_WINDOW_EVENT_TIME_WITH_OFFSET_DISTINCT_SPLIT, + WindowAggregateTestPrograms.SESSION_WINDOW_EVENT_TIME, + WindowAggregateTestPrograms.SESSION_WINDOW_EVENT_TIME_TWO_PHASE, + WindowAggregateTestPrograms.SESSION_WINDOW_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT, + WindowAggregateTestPrograms.SESSION_WINDOW_PARTITION_EVENT_TIME, + WindowAggregateTestPrograms.SESSION_WINDOW_PARTITION_EVENT_TIME_TWO_PHASE, + WindowAggregateTestPrograms + .SESSION_WINDOW_PARTITION_EVENT_TIME_TWO_PHASE_DISTINCT_SPLIT); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java new file mode 100644 index 00000000000..efea9082c0d --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateTestPrograms.java @@ -0,0 +1,538 @@ +/* + * 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 { + + enum DistinctAggSplit { + ENABLED, + DISABLED + } + + private 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") + }; + + private 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") + }; + + private static final Function 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") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA); + private static final SourceTestStep SOURCE = SOURCE_BUILDER.apply("window_source_t").build(); + + private static final SourceTestStep CDC_SOURCE = + SOURCE_BUILDER + .apply("cdc_window_source_t") + .addOption("changelog-mode", "I,UA,UB,D") + .build(); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + private 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]" + }; + + private 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, + DistinctAggSplit.ENABLED); + + 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, + DistinctAggSplit.DISABLED); + } + + private static TableTestProgram getTableTestProgram( + final String id, + final String description, + final String aggPhaseStrategy, + final String windowSql, + final String[] beforeRows, + final String[] afterRows, + final DistinctAggSplit enableDistinctAggSplit) { + final String sql = + String.format( + "INSERT INTO window_sink_t SELECT " + + "name, " + + "window_start, " + + "window_end, " + + "COUNT(*), " + + "SUM(a_int), " + + "COUNT(DISTINCT `comment`) " + + "FROM TABLE(%s)\n" + + "GROUP BY name, window_start, window_end", + windowSql); + + TableTestProgram.Builder builder = TableTestProgram.of(id, description); + if (enableDistinctAggSplit == DistinctAggSplit.ENABLED) { + builder.setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true); + } + + return builder.setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseStrategy) + .setupTableSource(SOURCE) + .setupTableSource(CDC_SOURCE) + .setupTableSink( + SinkTestStep.newBuilder("window_sink_t") + .addSchema( + "name STRING", + "window_start TIMESTAMP(3)", + "window_end TIMESTAMP(3)", + "cnt BIGINT", + "sum_int INT", + "distinct_cnt BIGINT") + .consumedBeforeRestore(beforeRows) + .consumedAfterRestore(afterRows) + .build()) + .runSql(sql) + .build(); + } +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithPartitionKey.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithPartitionKey.out deleted file mode 100644 index ac07c37f332..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithPartitionKey.out +++ /dev/null @@ -1,373 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 1 ], [ 2 ] ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, c], metadata=[]]], fields=[b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[b, TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 1, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 5, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "SessionWindow", - "gap" : "PT15M", - "partition_key_indices" : [ 0 ] - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 1, - "isRowtime" : true - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } - } - } ], - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[SESSION(time_col=[rowtime], gap=[15 min], partition keys=[b])], select=[b, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 6, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Calc(select=[b, window_end, EXPR$2])" - }, { - "id" : 7, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "window_end", - "dataType" : "TIMESTAMP(3)" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithoutPartitionKey.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithoutPartitionKey.out deleted file mode 100644 index 8fc9b4a3b35..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindowWithoutPartitionKey.out +++ /dev/null @@ -1,355 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 2 ] ], - "producedType" : "ROW<`c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c], metadata=[]]], fields=[c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 0, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "SINGLETON" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "Exchange(distribution=[single])" - }, { - "id" : 5, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ ], - "aggCalls" : [ { - "name" : "EXPR$1", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "SessionWindow", - "gap" : "PT15M", - "partition_key_indices" : [ ] - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 0, - "isRowtime" : true - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } - } - } ], - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`EXPR$1` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(window=[SESSION(time_col=[rowtime], gap=[15 min])], select=[COUNT(*) AS EXPR$1, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 6, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`window_end` TIMESTAMP(3) NOT NULL, `EXPR$1` BIGINT NOT NULL>", - "description" : "Calc(select=[window_end, EXPR$1])" - }, { - "id" : 7, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "window_end", - "dataType" : "TIMESTAMP(3)" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`window_end` TIMESTAMP(3) NOT NULL, `EXPR$1` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[window_end, EXPR$1])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindowWithCDCSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindowWithCDCSource.out deleted file mode 100644 index d310154854f..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindowWithCDCSource.out +++ /dev/null @@ -1,474 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyCDCTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "changelog-mode" : "I,UA,UB,D", - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 1 ], [ 2 ] ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyCDCTable, project=[b, c], metadata=[]]], fields=[b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[b, c, PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[b, c, proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$1", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 2, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : true, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[CUMULATE(time_col=[proctime], max_size=[15 s], step=[5 s])], select=[b, COUNT_RETRACT(c) AS EXPR$1, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL>", - "description" : "Calc(select=[b, EXPR$1])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out deleted file mode 100644 index 6a6beeaa383..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ /dev/null @@ -1,473 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[b, a, proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$1", - "internalName" : "$SUM$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "HoppingWindow", - "size" : "PT10M", - "slide" : "PT5M" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 2, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[HOP(time_col=[proctime], size=[10 min], slide=[5 min])], select=[b, SUM(a) AS EXPR$1, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "INT" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` INT>", - "description" : "Calc(select=[b, EXPR$1])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "sum_a", - "dataType" : "INT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindowWithCDCSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindowWithCDCSource.out deleted file mode 100644 index c7dbc5ac3d5..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindowWithCDCSource.out +++ /dev/null @@ -1,474 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyCDCTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "changelog-mode" : "I,UA,UB,D", - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[b, a, proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$1", - "internalName" : "$SUM$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "HoppingWindow", - "size" : "PT10M", - "slide" : "PT5M" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 2, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : true, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[HOP(time_col=[proctime], size=[10 min], slide=[5 min])], select=[b, SUM_RETRACT(a) AS EXPR$1, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "INT" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` INT>", - "description" : "Calc(select=[b, EXPR$1])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "sum_a", - "dataType" : "INT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithPartitionKey.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithPartitionKey.out deleted file mode 100644 index 059542e81eb..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithPartitionKey.out +++ /dev/null @@ -1,460 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 1 ], [ 2 ] ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, c], metadata=[]]], fields=[b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[b, PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[b, proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "SessionWindow", - "gap" : "PT15M", - "partition_key_indices" : [ 0 ] - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 1, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[SESSION(time_col=[proctime], gap=[15 min], partition keys=[b])], select=[b, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Calc(select=[b, window_end, EXPR$2])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "window_end", - "dataType" : "TIMESTAMP(3)" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithoutPartitionKey.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithoutPartitionKey.out deleted file mode 100644 index d493e0c3f3a..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindowWithoutPartitionKey.out +++ /dev/null @@ -1,432 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 2 ] ], - "producedType" : "ROW<`c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c], metadata=[]]], fields=[c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 1, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "SINGLETON" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[single])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ ], - "aggCalls" : [ { - "name" : "EXPR$1", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "SessionWindow", - "gap" : "PT15M", - "partition_key_indices" : [ ] - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 0, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`EXPR$1` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(window=[SESSION(time_col=[proctime], gap=[15 min])], select=[COUNT(*) AS EXPR$1, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`window_end` TIMESTAMP(3) NOT NULL, `EXPR$1` BIGINT NOT NULL>", - "description" : "Calc(select=[window_end, EXPR$1])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "window_end", - "dataType" : "TIMESTAMP(3)" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`window_end` TIMESTAMP(3) NOT NULL, `EXPR$1` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[window_end, EXPR$1])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out deleted file mode 100644 index 5a0ee6c4f2e..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ /dev/null @@ -1,459 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 1 ], [ 2 ] ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, c], metadata=[]]], fields=[b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[b, PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[b, proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "TumblingWindow", - "size" : "PT15M" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 1, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[b, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Calc(select=[b, window_end, EXPR$2])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "window_end", - "dataType" : "TIMESTAMP(3)" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindowWithCDCSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindowWithCDCSource.out deleted file mode 100644 index 5d5fd1c39ec..00000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindowWithCDCSource.out +++ /dev/null @@ -1,460 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyCDCTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "INT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "TO_TIMESTAMP(`c`)" - } - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "serializableString" : "`rowtime` - INTERVAL '1' SECOND" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "changelog-mode" : "I,UA,UB,D", - "connector" : "values" - } - } - }, - "abilities" : [ { - "type" : "ProjectPushDown", - "projectedFields" : [ [ 1 ], [ 2 ] ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - }, { - "type" : "ReadingMetadata", - "metadataKeys" : [ ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" - } ] - }, - "outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyCDCTable, project=[b, c], metadata=[]]], fields=[b, c])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[b, PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[b, proctime])" - }, { - "id" : 5, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[b]])" - }, { - "id" : 6, - "type" : "stream-exec-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "TumblingWindow", - "size" : "PT15M" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - }, - "timeAttributeIndex" : 1, - "isRowtime" : false - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], - "needRetraction" : true, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[b, COUNT_RETRACT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end])" - }, { - "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Calc(select=[b, window_end, EXPR$2])" - }, { - "id" : 8, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "window_end", - "dataType" : "TIMESTAMP(3)" - }, { - "name" : "cnt", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/plan/window-aggregate-cumulate-event-time-two-phase-distinct-split.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/plan/window-aggregate-cumulate-event-time-two-phase-distinct-split.json new file mode 100644 index 00000000000..94c76a50b0f --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/plan/window-aggregate-cumulate-event-time-two-phase-distinct-split.json @@ -0,0 +1,908 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 138, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 139, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 140, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 141, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$MOD$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$HASH_CODE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1024, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, MOD(HASH_CODE(comment), 1024) AS $f5, rowtime])" + }, { + "id" : 142, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 3 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 4, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name, $f5], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s])], select=[name, $f5, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 143, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name, $f5]])" + }, { + "id" : 144, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 1 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 6, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` INT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name, $f5], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s])], select=[name, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 145, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" + }, { + "id" : 146, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : 1, + "windowEnd" : 2, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "LocalWindowAggregate(groupBy=[name], window=[CUMULATE(win_start=[window_start], win_end=[window_end], max_size=[15 s], step=[5 s])], select=[name, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, slice_end('w$) AS $window_end])" + }, { + "id" : 147, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 148, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : -1, + "windowEnd" : 4, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f1` BIGINT NOT NULL, `$f2` INT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[CUMULATE(win_end=[$window_end], max_size=[15 s], step=[5 s])], select=[name, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 149, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f1 AS $f3, $f2 AS $f4, $f3 AS $f5])" + }, { + "id" : 150, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, $f3, $f4, $f5])" + } ], + "edges" : [ { + "source" : 138, + "target" : 139, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 139, + "target" : 140, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 140, + "target" : 141, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 141, + "target" : 142, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 142, + "target" : 143, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 143, + "target" : 144, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 144, + "target" : 145, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 145, + "target" : 146, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 146, + "target" : 147, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 147, + "target" : 148, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 148, + "target" : 149, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 149, + "target" : 150, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..0312d662ba0024fde427811fe999535ce5235251 GIT binary patch literal 29761 zcmeGlZHybmb!^{VKFM7mg%Cb;G?zdi{{C)2;JY}9!MVHIcb5{Rn7Q@Z*~DHuyX(74 ze!!)O282}Tm&#WmrD+8;6eJXtRxNE+R8;jNEmZuVh*oW-RsmXCicnk7^}X3$dwsSq zJ`$T8wS8_RX7_*sWPRA*2I-=q##7R{VHo&s+CB_x4qfz1nr?p*OD> z(&DBuUQdM226yA9AA9vc_`uAczrOM(+&B090HQGEw1xEgsNYRJX&Q8U!$Ef@!+WXQ z#c}R{m-_wuI0wmAYRSQ4PA|lQs9DJBWT7N+e1fNGUP>xd;k8s>$Ot+9Q_cx0IWvVE zM9R**+nEY@_WQyqx9iXWe_(&cbHG~^v)P~vUG{);#JR5|3X0nm^7=g@bb>Dy1W~bveV$ApU>|o=H}4Pgkdt$c2mDYKK4bTAoX_s_Lwtw}hwS448uGZ^ z=`bHm17fG(*VPbhtQrtyXgHdPj0`1G$&tuNG_^IB97)9bM@M2qgQ=ZETce~WIUJ9T zR3f4*U(Z-1J{nCWqls7~9@`yFK&X{S1u5&GMau2v9hsaU?uQq_!4>lPg6L2V7WtAQ zE|G$$P5rM|f$lOiq3dkYGbYlB~vTYl}S zldi+}Is)8-qmM@3nSn^g1DL3)MXh9w3|QJFUXduLfG?MFij3)&;vXSbe@XLCs{S=A zG=ID1-=X~)y8UAO`+pC-XWDl zMac68Zi`MSyd=_`fw5|Xs@Ungzq0Or{4LTcOXQjcB_Th{j%@;;OUqlSLPy~J-_IPs z^r^#-rmVygC#y0Tr$@;O>51>76V!>Fz!?{0Wy_RAHp~fK&-}A{-&ymB^^UxiSmUId zD~J;?#0Uj(AV*~YyjlgEqhi*{g0{rTid-Qpa5RT#DZ~?pa-3?{bh57LWb|^fS~#_| z&UhuymZ{=|MAp@!$EuwuOsZT-Y*x|%rLns99~`84D6QL2(Veuo_f4!T&b6Uw~_Qb5X8MGcf*=*5OYof>1J#tnq|plakciTLJ=66V#=BIgqVwOWO&=|B#0M8#6x7+F+^EitqU zXt)wgzk7rG#G9Y*Vbtr$L5NPf&8G)Y0eo>k znf}F|_bz8(ESy3vG_-IfncBP;R3Niy1pC+2;g>FZ;6?g18?nSlhYWKHkPF7DN$lQT zP#=F&bklgE5BzP|qcz$9bJ&3K)4a?{LQ!Sv^9?+mR3t&nB2SCZDUbq}z%#ZuK6gwN zfvzkH5NVF5#;7!~mrAo3G)B5|yqHz?P9fa$EVrT8qmU(Ik;K4`NaFUHnHk2s>!84u zlhAt#lPOuDk^%*Lnxeo6jHr+!U#u?L8Y9b!63+=Z6qWm~uQ^UG!({z=7Vejf#CAqO zBH1~d+7%nwF*G`o!jRPNp+S^CY{2vL#2ib}8jf*15@|IMDG}X18jmE(1X`{Uh!@i= ztj~feacZ{V!`0=4HcmtWI-4S3zmScIN;UP!tU7n~$F|1?N6G~(kxTg$h=5Q?b7R+O zOIQNnquZmE%C1a9FA+peNnvHMer{G>FGKBO7P1zqUgml1kEE}8H;{}-1pa>or zXl&!YZ#|KkajoI1>9PxmlVfc&} zpx58?m-~Ks@A^N$&~+o}oS?Z94`ZyM@`I)PI4^zf_*X7l@#R0>Q>na0e^wY)e07Ch z(7b$>m(Ksw(PQr&J`w`ZF_ILPt@8S%nueheOBk9xBbvS!x@(4eYBHqwB1Yh@@H z`mj|EB2-OCl>`MWqe)L^XJl)3glyZaYiY-AwQ1zq+PH>-#S;f7LBT(pt)l{8#ar^b3@6t@j>Y4xk2 z4mL7452zq)8-P~5Rl#fxrn6PYrUDhE08@dQ?muWMP)!A@V@?fhMKfwDP}wq&?9jFQ zrUKPeplS=#>T02a^GcaLOa-c`Ks6Pp`l2z&{ODkrs};+ZZPDA93RE=0Io(SCV$h_S zdUIwBVk%JU&HMFNXBVdq&{Uur))42b|1%Y+u-xuA0|J@~RMkSZfl9WoLAXIBsj6yQ zqn2Da3RDzaL->b|s@`Gp@UOc|K%cvwR(-SrL$_S@z<{;v8s_c!shSt;U6}*pIpz#D z$mP7_YE-#gJ+EHQGA*zE4zCUK-v5ePuAX+TytC`8=baDEdY;}>YPrZ(=yaOt?+{dj zv!3_y4?cWe*Xon3t@Lu{DtC5$&GuzYWweWL*hc4Y`>K^-|L<-edt=qNgRf+Eu6#-A z_&FNRBV$Zsq|PDuY;b?ywR_Xszqu}#e&B{L4zJs_5u$3cXuCR=O@i7(%2R1SrqJP? zXt%XJY~J9<-RYp6!Zr}Q&l_;rX*!g#yE5tEcqTx79zO;Aeh5G4$re&UTf@4U-5i~x zgRrxMWvIV}qct-J7Fg;&dFkKafh@s4aMSGH{4*(FQeaV1pxq;^Ki~7ND`Qh&z1p{D(1MdE25 zV6p9!g(5Fer69=;{Q8qBW+{fRJ{wW@tic|#gb$ z`~)v5W@o6g$Iu+g>uoXo$cFOPbcPk$G(fXiZNE2svh~~)y*av7+iE{@^Kg`OvVHV0 zOIRDs4y}|xY^I^Fwn2KnihgsIF(}ct7GW3ckgnpIN3y&|Ge1w+U>X8dH++FDBp2Lx4h6qmX{y;9e>l^ z4<9`7&@-<-{=|`W^|G8>I@7c}9Pqh)etUrPcE;yvK(V?h#Xc3%A zN+$$<(g6oikAZK{J+fn|9Sz7<&WEQ(S15gA`1?0L%N>UU#_`bR{+i5<&q=Cou_RcR z<#?qa9(>}+lP@|ZqbfjOa{#MhRT>hGa5d;eIu`w+2ztmZ1<>d^kv}S4-vUae?h+keoYsl-N;5PGcyd5{q?FUX>h?91>NC)Ao|SBuF~re zx>k!Lb0n`@CEcXc=vP$|sF@Cyxtwo$nA5Z=4>cR>{7y~dDaX&mmb!4MrxvLqMbW*rRb z$`s%#MR-_N5O ze)P!8OOJhK&2#K>r{#cSp0ay;6ymSF*N7`fypx~eIZ%ofL}wY$wsJQ8`X`0dy&7r_ zTw|i)KbWf+M?>!+K6`_91L6o31|WLlz3~1fOtE~csuIgt@BeemxC=Ehzt%5 w#s(sBP%vsO+*^hT2mm=|T?<)<70Vg073$W;h!b9Ivkb1uNRB3lqk~)j4}F#op#T5? literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithCDCSource.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/plan/window-aggregate-cumulate-event-time-two-phase-with-offset.json similarity index 64% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithCDCSource.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/plan/window-aggregate-cumulate-event-time-two-phase-with-offset.json index 24b1357e2f2..b645f6a8e84 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithCDCSource.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/plan/window-aggregate-cumulate-event-time-two-phase-with-offset.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { - "id" : 1, + "id" : 159, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyCDCTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" }, { - "name" : "c", + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,19 +85,24 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "changelog-mode" : "I,UA,UB,D", - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { - "id" : 2, + "id" : 160, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -94,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -104,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -117,10 +134,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { - "id" : 3, + "id" : 161, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "CALL", @@ -148,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -167,19 +184,19 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { - "id" : 4, + "id" : 162, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" }, { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 1, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", @@ -201,13 +218,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -218,9 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, a, c, rowtime])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { - "id" : 5, + "id" : 163, "type" : "stream-exec-local-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -255,22 +272,14 @@ "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$6", - "catalogName" : "`default_catalog`.`default_database`.`concat_distinct_agg`", - "class" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatDistinctAggFunction", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "VARCHAR(2147483647)" } ], "windowing" : { "strategy" : "TimeAttribute", "window" : { - "type" : "TumblingWindow", - "size" : "PT5S" + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -280,7 +289,6 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "needRetraction" : true, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -291,8 +299,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "count1$0", "fieldType" : "BIGINT" @@ -302,50 +310,6 @@ }, { "name" : "count$2", "fieldType" : "BIGINT" - }, { - "name" : "count$3", - "fieldType" : "BIGINT" - }, { - "name" : "concat_distinct_agg$4", - "fieldType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatAcc", - "attributes" : [ { - "name" : "list", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.ListView", - "externalDataType" : { - "logicalType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.ListView", - "attributes" : [ { - "name" : "list", - "attributeType" : "ARRAY" - } ] - }, - "fields" : [ { - "name" : "list", - "conversionClass" : "java.util.List" - } ] - } - } - }, { - "name" : "map", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.MapView", - "externalDataType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.MapView", - "attributes" : [ { - "name" : "map", - "attributeType" : "MAP" - } ] - } - } - } ] - } }, { "name" : "distinct$0", "fieldType" : { @@ -373,9 +337,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT_RETRACT(*) AS count1$0, SUM_RETRACT(a) AS (sum$1, count$2), COUNT_RETRACT(distinct$0 c) AS count$3, concat_distinct_agg_RETRACT(c) AS concat_distinct_agg$4, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])" + "description" : "LocalWindowAggregate(groupBy=[name], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" }, { - "id" : 6, + "id" : 164, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -388,8 +352,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "count1$0", "fieldType" : "BIGINT" @@ -399,50 +363,6 @@ }, { "name" : "count$2", "fieldType" : "BIGINT" - }, { - "name" : "count$3", - "fieldType" : "BIGINT" - }, { - "name" : "concat_distinct_agg$4", - "fieldType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatAcc", - "attributes" : [ { - "name" : "list", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.ListView", - "externalDataType" : { - "logicalType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.ListView", - "attributes" : [ { - "name" : "list", - "attributeType" : "ARRAY" - } ] - }, - "fields" : [ { - "name" : "list", - "conversionClass" : "java.util.List" - } ] - } - } - }, { - "name" : "map", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.MapView", - "externalDataType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.MapView", - "attributes" : [ { - "name" : "map", - "attributeType" : "MAP" - } ] - } - } - } ] - } }, { "name" : "distinct$0", "fieldType" : { @@ -470,9 +390,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Exchange(distribution=[hash[b]])" + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, + "id" : 165, "type" : "stream-exec-global-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -507,29 +427,21 @@ "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$6", - "catalogName" : "`default_catalog`.`default_database`.`concat_distinct_agg`", - "class" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatDistinctAggFunction", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "VARCHAR(2147483647)" } ], "windowing" : { "strategy" : "SliceAttached", "window" : { - "type" : "TumblingWindow", - "size" : "PT5S" + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 7, + "sliceEnd" : 5, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -559,7 +471,6 @@ } } } ], - "needRetraction" : true, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -570,13 +481,13 @@ "localAggInputRowType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -587,22 +498,22 @@ } } ] }, - "outputType" : "ROW<`b` BIGINT, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT_RETRACT(count1$0) AS EXPR$3, SUM_RETRACT((sum$1, count$2)) AS EXPR$4, COUNT_RETRACT(distinct$0 count$3) AS EXPR$5, concat_distinct_agg_RETRACT(concat_distinct_agg$4) AS EXPR$6, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 166, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 5, + "inputIndex" : 4, "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 6, + "inputIndex" : 5, "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", @@ -616,10 +527,6 @@ "kind" : "INPUT_REF", "inputIndex" : 3, "type" : "BIGINT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "VARCHAR(2147483647)" } ], "condition" : null, "inputProperties" : [ { @@ -629,10 +536,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>", - "description" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 167, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -643,12 +550,12 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" }, { "name" : "window_start", "dataType" : "TIMESTAMP(3)" @@ -659,21 +566,15 @@ "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" }, { "name" : "distinct_cnt", "dataType" : "BIGINT" - }, { - "name" : "concat_distinct", - "dataType" : "VARCHAR(2147483647)" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -685,61 +586,61 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { - "source" : 1, - "target" : 2, + "source" : 159, + "target" : 160, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 3, + "source" : 160, + "target" : 161, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 161, + "target" : 162, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 162, + "target" : 163, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 163, + "target" : 164, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 164, + "target" : 165, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 165, + "target" : 166, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 166, + "target" : 167, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase-with-offset/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..677c5ee607f39d41a472686a84597bc6972c7ad9 GIT binary patch literal 23185 zcmeHPYm6J!6~6XvHrYJb4TKUNL8;gP4e|KxwYzP>yKxeOv%A>dO$ih;9*@0~wP(CD zgR4RhXqZGBJ5^X6;&$%;W z&#cGUN8%-r8Ogpg_jT?)=RVH)?wLv6Rq_x*I^lnRpPVNb~C@ApK*(SA=T77m4rg@`}T_Cw3whF@39c4OawD#%O z&wv!=3M)@){J<=PoOF>D7HK3J@bw3HABeOV@q~i;sD}*%c#n_u`C`F9%wLSgpkY6Q z9~Eh%-4L@_QLY*r2C>ot_VOEDg8WjFSTf1V`Z!-Yu3FCvSG@_8$!h$AYmWZ6?eD7B z+g7zxvBQD3b70G3Z71;&6Ik&jmgj;+U)1mS6vHvtEB$Q1laGaBo(|XtxGjB_q_aaJChVC5`C>SA-bi7QFE%Dh-oCuSF0PfDxLT911_St}^RwQ?DImm+JB zc3P5Dl|)j|Jg>H+d+Cnur7ZB$=6Utzc}>+~xq;cL>u#BZ7S*+RW%cfniwS{VU37q* zW|=Z8mY9TCtC$n3Dp@@qPY>;hr|&s^`ZU?J??Xt|w!-}_C^ri(qsoF<;bJKst2|#M`t3Xs3JC7di_>3Bn>=L(u)|G0wt7d`U~+eIIIHt-$D;V>Fb_34RN8=ZiQS2`Gx|WW z$N7XBvpy*nrS&dn^rhJu>(q6p$SSVP9y+9wjVWB2y{Ke+hsvy?45Y|frH;bcs!;&z zc{Ny%S^^Z-^CE?Y$}w<(E)E^(_`YeG+|XiGKtQwrrsA}~&oF!0>Ntdk^dX$Bf=$S@ zC|fUwwpxSJCr6*U`3des&P`3G!KT8kF<7g_GGKF*c~ue*KKj7-pJQecT7}JKoNa~H zx*DrIbjf6vbdvULxWh~6H7vyq=Ji>gEJjs(T6$+wOIv~p?94b@uJQA%Q<%#((Uz8L z<{&Zg-9-{D=(ZSCebzB%seaRNd~YH%8Xsyp0B8@(HWUO!6+}*rb6gJ3Y4Ebsr#kk= zML= zmLQ3AXRzNa*6A=1l2l1x$&eAiMX+FA+oE|mKA8q0LuIf1(Y11M`lo}REr_3)#~}Ff4-QYQ9OGaIP&_@=O4P}Re*(WCab1#-v9-yw_$V-*D85lzW>CxwygW+ zpO4^@fU3qP%J?HI0XpmIw{>%E&%U?P`|M`RKARIk%hUjFebKesaucr| z@=DzEIWOL*PRMCkodDA$PY?+y%#ev?Hf19Wh85*_t|X z3Q~IOEo#nY+Y!TY#mJ6SEFHlUxGf7+y+$vNO*gJz-TKgLwb9UPpWgN%_m88ik3u|F zK8NW|eZffByZ63w-5bfLpZVug@mHSt<&5FCHiu~~iv0fA!*5-AI`i;f-v9Zf%)8rB zPzzBRsNCV3PTusvJsaP9_Le9Az}OX`u5obi2#24O9+_8VmQx{AYGsuo-+G6)e*>Ea zl72x8Z&;^?yY=usJ^Y0pexQXr`pJ5|>iJdf(2?~WKhU%ORSP}W=^>|wIK1KV{80;; z%U0a=ARO_9V>}o0aG{vb6XJqVPd@Jpdcxswe~6Dn`|~VJJ0=4S1lNPs+(dz$YoERI zWq;l!@a2MHn4$mr;fbprJNyV>TCNnisEBDslA@Ac4YEYo(VGHn0I|mN+`PS_*}SC| zJNSuc!;*BkNoR_zqkSDbILubSnC>xk6cCAl+ewd}J}N2TGbxEGx#V^efIE^uz;h}f zxSf;Wh@g_KHr6=O^#a{nAP3~UsH#_G5WU<)GzHU|FYIBJ2_#QVhQ*zCK>dgZ=zT+P zl@#fsZC9qq-NbeNp~EYpQgoz8_JP@TEw#&V{4zsQxeOu$Q7V07Vnr$lMS=R^z_|Wj z&o=+bcOLAe-0LhujKRAd3s=yn`0_zA_v<^~znWI#;#Be}lY|@DH1S^2sG0iX;{2LD z{Nk2xKgWKj1AUG<6>!c4P#4y!8Qk-_pg-Z}(QR||t&fR52zTL(ZVZkN82ANV;bftz z4eEFZDUbtK!ac%Pc;7i&g;OP(1PwVia1hqJ!F4wVy;e6PI(V-Dg6p*t zUbK&76$l3w#R2r;ffALfb#Azmq`NBfoPbDMeeQ;A<}Pw|p7T|`PCB1wt!=+Dn*^vf z8{a#c+n3Dl85zswFebNuWEdT}JHTRIV#|?=E;xEzh4eN8lhTRZW2tz$&Y-nAgLtt( z|nO5X%n`hcN}fN zjZo||I=j(3jY&yhn8K#Y;{Y1J_Q3ZZ&7Ec_?Ar;LD{MZOYI6`XgNL7<8K)t}*d${; zxP~BYTWQ~jDSOV44h=jPuojZh`Csq&M%r=eNeJTQOT2vXYsZeifB1nYtcpnkF~gF> zv`jx`UQ8>G(IfQM5UUmbX{Vb;mk#`&o0V;`E9583VZ-bwskCsNE3+?k zHTES(>{RX7kLo@ukL{02>}R7~T$mGdIbKFKdeeo;JVQU!7aeExEz`%%bSB-^_A}|b zT3B$+J{+j%QY#K2sfL$GUyFDX@Qwgp2U$5v?KzkH_&8h5$BrEU6}12dKrIX&bO6+I z{m!2Qpi+F?0Z=FAfskhNo1!QVfT|;?_)Y|ZOAdhQ0I2Xbhy$S3KOEE2DSpXs+B*QM z{t*Earn^*o%)Ej-NZN&rq&c`b{UV10pgJ!Jn~o}NK2G8QsAsd^I4=pqSAWiefyP@3 zjj!T30P1p}+6x^(MLRXB8>@uXZVtcs$ZvXT0Hs!+&_eVA!)vV;fCFp&s+R@-R=nzE z|NX31*2rr(bsNu+w_bKWU85}hOc!JXE1F&%dI&+i?0mX&FE5|2QFbYGjWTQA)#xGu zjhDeQ;~Aow+H90P>Ux3l+sM0=vYRhmeA)SYcH~P%RsV+9nPhYx%UA0J{r~aW&pmeI zOAp?A>w91L;NJUw`#416pqj%(V|3xY>H5=KU;9t*mwt55{h!}3b!x0*6C^EEM7I#5 zO$TbmQ5t+7mYbyCw`GRvx1%zC>r54R-LIdx-DJ&v(hJBC$I68vR^)hal`ZTi_&zvB z?PB4xyYSp&JPV?rOqgRbwLCcgVT@5trlwTEB@v^#iV}Ry7jC+1B76%2z?TAeLpR!n zK0mhaXX@SXHnMt+Gd?T>K5&$OE&^_i{>3mMV|?xxiywR7r8URDxamnZ0k7D#(8dZ& zzq_N7aI-@wC&|2*pXE95C6+{Qy`o+9Vus(WN^E^Co}-q gG4I2~Hbb#q0Ht8t8*`fww(SQ0qRd!kG%>vM|L_G-mjD0& literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/plan/window-aggregate-cumulate-event-time-two-phase.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/plan/window-aggregate-cumulate-event-time-two-phase.json new file mode 100644 index 00000000000..7390785ec6d --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/plan/window-aggregate-cumulate-event-time-two-phase.json @@ -0,0 +1,647 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 129, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 130, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 131, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 132, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 133, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s])], select=[name, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 134, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 135, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 5, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s])], select=[name, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 136, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 137, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 129, + "target" : 130, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 130, + "target" : 131, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 131, + "target" : 132, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 132, + "target" : 133, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 133, + "target" : 134, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 134, + "target" : 135, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 135, + "target" : 136, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 136, + "target" : 137, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-two-phase/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..7bd1c4f9eab1e3397cf789a7c302d9ee18d6f7ed GIT binary patch literal 23181 zcmeHPeQX@X6`wnQ5XT92NLmOG6bUD7gYU!V`>>m;^lV>}gNtq5otd&$|!nLu`B-|WuJn>X`j-prfdyxsWic@H6^9scoh=>b{y>K*UzdFjPp-8=i0 zF23@C>m*A%is?os7gD)M*8-=BN|96MFu_QC%0=$WNd@tx zBw6ze1O?F_^bGMs;Se7hh=fC7-+-S78h*$#$ngW7U?dpv3=M{P4;S$d3VuI79Lb7M zzh~jU(^_p-YZuu*F&Rrn(-X-|DjiM7GGp;nIvF3CO2;S0Gy5mTVx(hg9PTMW?{qXV z70aYz$#^sopNS~0C>4t(*{dy;#fl~sR4>n|qBl{J z^C?l0xPo+8RQjxsDVZy)vn35mZG(T}s`R{k?VEUa8UDa^l>WBt=h`<~Rd`R#F#Ub9q=eUq(Fd_#0e!)M?`-cd*ZWhD>Gc3-A z>AKla3!I$y#^g%T zoLJV#Wz*5*=)P$3%V*AePAnuKBZDJA$*3Aw z55BTe5=2#nk}?aDoGmSQXsQYoqN^XRnEi_Tg}SP$nj&&VD2uMWrLw4ST1ip8X=GR% zVU`MQ{5u*mZj%t4(d&%f27{$FM&5RdMu_vGtU1a+mkT;qI#c<@IhD6Y8M@grz~%F} z2;k)+y*I~hiH)(%S^Abq*pJao7$mxh{p!$43#6S{Aila4NVjsV!*)c<$7L;TF+gyI zZq9x&E%H{CKj@_5&D6)NJg-6`m@O|+%v`gV|*<8Z^(}^PepbyYDQ5d6pnOLP?&7gAb`B)0M$~vFK(n z5VF)rV8xITz(p`&?rcy!9G~7ABExzg13FhWXMZ~Q**Wnu(|C4~Rs(2`zA)o&_4;Ch zM)Bxv_whIGed(U-Ujtb9CeksF>joHLy|qpEc%{gT%2!W5u&w*se?E>=0-72htIZ!- z3(%RF-`2&oHKxK^ud`QK*4c~F z`ksb|UZ&%O*KdviLtoM<>AP?JZ1dRb zZ~dkFj&DH}B>^X(NLUO6gF-lx9p**&8yE}*#eg_0f`?H=7!E+z4JIMfl(l7Y<=Zy$%1= zBg(eg*tBLmqk2=5iFmr^G8LnA_Mp5{w^M1njpP)CTk?)9Y2u_J6(voY7o!+e+gtXX zEzzjxtUWNd#?Oj>19ZxV*v9QJ!;Mm_g{}K{pf4Qc1|z#|KS(E zfA`fS%Xt*DH6Z!?H~zHv%Sk@PNhh`EI+AxcLDF&RwVBZUuMAfny6*+)ukTDXLGthS z9{yK<+g5M-ofi+E*u43O9SI$saKsJA>~kJP)D%w8AXF;_jh?YE)!KDG5W}?#1p?O% z-7MV8!aG^`a~A$j5Bnoz11oxYk^Srpr~e6-^&fiZ`4kHU7CyqlKkA|P%C+`!;dA_O zE)dCivi@+$6XZnE!$%?_I28)Ha4t8T75E$>-W1dibf&Q|TcI`=iy^p25(`+FKJcnsUlB$tj9UDh{+m`?k3PJ6Q97nmjZg$cE9&kv~Ml`y>pyn^wSu>fkfXo8?fvU$F)l$PpEUJwEt}rGn`0d%E34+ud>)+Kr|U_3}zmn%^jrOv6aSL z1x>GM%xVt|uxvNr@+JsT*N+qLK00)G_md+!goxR$qgB|BZkV(JHv-v}RJ+j)5?&Ml z1OygU5eLxp&V%27EOW+7&llPO+=s>I6S@v!W^g5CnNbZf#wHod!8rtZ+xGU2n6mFI z=}^H7fuAB9CjV-~H_?VumsRLqF)u1tynf=*_mAB*1hZn?fM$_$oYF+7@P>7IjGmyk zMz>mV##Z_Rb+Qwt3p2AdJ41fFJcG<0$xNIpob_5aUtgL%vNr2dXKh__x}BQ+>`eH> zdNts~>h|+7E-p<4U5z`Njoox<3eT_)*`(u)zEwI?nw%dxTkiaDdxITZy|WM+x^x$Z zkdz5BZV+!mixI$UBWuU0H3#4y9k0IS(6Ixcq7~o(sM(Pv2SB~hFPAz1D#gbg0Cjd5 z2x;DaQxwGkP#J=XFW(@z8^A?W}d+v zB<)g0(j45Je)`1$P@T8YO-GeBU%_zz)bm+yoVU^8^HS#_K;w0o+UJ)X0ChD`?WGQ& z;yx8VzouWD9vmaz`%O;;pwzpc&_nbB!)v`BfCFpws?t{9UtqMo&*`*AUgPZn;~7e* z(k|y~(9-Y0K}Og`)2oez5L9WG^Zm{%tLJObu7s~av;A$IFCx%*88S1TA*!y<2JH_{ zub$Sw>1Eni2A{L>T`p&*e`!}$KNfekKDvneS8oLUWqY@wx}z@RkPE&rE6mZ);d)1_ zx04Ef?oN|L=5SBkZnAEM^a5rha7uQRlLZl+ShF((KM8k@ERqF3)#iHl!C6OLNgxYw{v50m0@mAG zt+&^v-o|)B@Y3S2XJZ9-jYh{O#^a;W1bn1`_3-v;nNR@`$Fffw+YH4j0rrAzYs_^* RK(z}ze^OJa$=LYV{{h?MOZ5N% literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/plan/window-aggregate-cumulate-event-time-with-offset-distinct-split.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/plan/window-aggregate-cumulate-event-time-with-offset-distinct-split.json new file mode 100644 index 00000000000..82863dbe6b1 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/plan/window-aggregate-cumulate-event-time-with-offset-distinct-split.json @@ -0,0 +1,912 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 168, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 169, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 170, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 171, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$MOD$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$HASH_CODE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1024, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, MOD(HASH_CODE(comment), 1024) AS $f5, rowtime])" + }, { + "id" : 172, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 3 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 4, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name, $f5], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, $f5, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 173, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name, $f5]])" + }, { + "id" : 174, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 1 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 6, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` INT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name, $f5], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 175, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" + }, { + "id" : 176, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : 1, + "windowEnd" : 2, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "LocalWindowAggregate(groupBy=[name], window=[CUMULATE(win_start=[window_start], win_end=[window_end], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, slice_end('w$) AS $window_end])" + }, { + "id" : 177, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 178, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : -1, + "windowEnd" : 4, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f1` BIGINT NOT NULL, `$f2` INT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[CUMULATE(win_end=[$window_end], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 179, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f1 AS $f3, $f2 AS $f4, $f3 AS $f5])" + }, { + "id" : 180, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, $f3, $f4, $f5])" + } ], + "edges" : [ { + "source" : 168, + "target" : 169, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 169, + "target" : 170, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 170, + "target" : 171, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 171, + "target" : 172, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 172, + "target" : 173, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 173, + "target" : 174, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 174, + "target" : 175, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 175, + "target" : 176, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 176, + "target" : 177, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 177, + "target" : 178, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 178, + "target" : 179, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 179, + "target" : 180, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..ceca38c16a7a654382567d10d8bb40f3460607b4 GIT binary patch literal 29761 zcmeHQe{37o9e;Mx^mm(2z)6~+We8yX;i27kb)1e%bx!F~vBh%wk;3^X=%-|xFS`_4YM zZrX&F+>`8g_kO?c`~7*J_bze62tx>IhaW1JrjdohuNGgp=HQ3RR=(dKTz~iK@9K8@ z(cW-`&n9)}qxawT$>qOz=%Vg^;gyH`J_0LrxgVVvUR)TFi_9NqRY5MY@@}*+ET43d zJM&^exZFF&%HDjbzzeeCE&9FLp1|%3bS}9oA&)sdkaIL&Nb@EHjkKrZcfjJi9TG&ZH9k z+cJrv!R*$djd3z(+aNqs51rd%$!+m$I-W|zl8K%16qvfna7iBVuw|AT6+HQZDD8$9 z(ZiLB#ggPv_m+iTUJI(DumhwKjNw^J-X&? ze42}Yp!U;$=Wmut;G+9xS%&MfdvUSOVAj6QbPpQ}g(5j$PGCX;7i2<#9LMA$xgg{7 z2YMoGAk-TT@Pu?>_Xs^K_J)~;-cV(81;Ha%BvmX5rVcSx8sEsOtf@O}FsbVi-ADdT z<|s1pHHiZ|Tg6i-jffmu@N8%0fl*e@fT3PrTyEk@kdA^Nji{p&t%j4-!n20s-bogS z%3!5Xz}zLtJd8V4Dyb^zNka0x+JNAt1A>=|z)O?o)syEn1&jXrCd#gTG6}VmwK~6g z_Q-NVAXiIvv16>Kz)B;YxKyb&9keW~WbXD@YG6w&^|fQij_Kmu&{CWUNwtaq<7%xW zOv$a4#Mvvs#9$`A83z9qJN^gR!P$?S>}(>##%iBdWly6m_p@uhKw+A5GiIv=5#wVwlhr6GBWDhq58 z!lJXGQdy8$wInN^3`ST>o+Uwx|BBN~w?(i{f8z8f+Ne?M*`5BxG>qDH9o=?@mZmlx z{mJw!gw|U6C}B*HRHu8QQz|;2aHc!XbjON5X!C7r!p$k}MdNnMY;ujRl_F@dRHC*2;1p?Yqu^nSFxvWXAiVJie z0Io4n824;t%iBTMiv_ibG9X!=CVs2_5MioG`pMy^u6mq1l66y{r@z5-b4E1p~8zX1W%H{fo@@Wc%vg6y?LPe-U zg`h3l7+cnH8e~FDa(9tI#Cd`4}!L%%Gtb-9GwKSMSPCw;p~jJ$KDNC(s&x z`_bM=13sJ7x~Cs{`o_`zH!gnnS>InC{4wB|C-m~(2fz2ey0>q`I|tied;ZInyISz_ z*5nWV%`gAuYthxW{psNAcfQktmmj{e@ABt`Yi{2B^}t70e0Ux58c&p$!Hu0qzkl`C z5BB}&v3uh;_CIIm<;-HF@)0h|N5dS$3tXNFMR}gd1%g2)nD33SJ$Zi)D65bQhmi_G zaFinfM*?TN1ZJ9%>I?XT{y>gpdUAnYprx`b)5`^U2KfE~E*}#7Q8r4*k}iY8>=W!r4$KRUHnr;48nJ?68jz0(6k&0e;h(-@q_0L*mr z8N9QB+tuc%TZ32)o1C(O+BE|{_Q4&C$aqk`C z&I_#saH0)P)$L|vL6uof1zo8WR7w}uw>8$^t)*o>#tZ7Qg}T03*KgAGXLbEkP2Uh9 zi}axDgWRCg(Kq~5cltL?XTGTGoUY@Z5Gdw%n(jGwrmH3w<^zI19|A~+4KpEso@Jt3 zpoa-ZqP}R9%LVe0JRzPmqz|fQb74+L?L1TVv^R-DfmaOb*1sM*vgVNk4*(C;l_X2@ zXlGa{D(Tc{zDU*eCV@|f{M<8~61KXb6VSM7a3Wl*m88v0+LL4<&1?VOLAEHUWT`2m zz~bw>jx5#fhf50d4kbw?=UitZJrCt}37iV7=l0!jz*Qw{Z5V9GuJb~?SAz?1xumQI zWkhR-I+Mb>_JeQs%9W?FA9 z#tiKvrO67yI(x|cD7#9Gc)5-M{Qhkik(jT z_2Vs{)!j^=Rk5GScwCx>qOgmZTh9)bx#&eH2(WR?6 zgrrQ!Y!h=6D2o7I8=1M7riVgt_8d^IhmIWp6=wksfXespbpX`UeY>Xvpi+F?0Z>Ql zKuGhAtQ19Y091Vmk6aT3mmC1q0Z`!vB?myQeqo^LMKULSr?vy2o?hCkgQU%7B+bFi z>28DrpgI?8nzkx!K11LDsHPpOlRh!$kgwtU9;YBc<7S`Q7aSY_bt+KpYzI(roeHOo zHN|S$fwv!cWoZSV)bit+jw~=#3p5P~tX0+2?F|<*Q|a{_AVEXv0>QvB?FW_~Uld?-6&C+P&thGS)ZfR#7_NUZ!$g14KZ(IytYk_hihjdxKJl~J=vx{?E0 z9-0`zq$^4p$bx9pkuSl45_p(jk>F-$fJJ!VDc915Y#iI$2K8)sYfin!?i>06b{Z9f zivZ!!---~@#^)K)`=MK2n|JuDD}L@Kz*k)WX)Lnzi%}{GH&QjZk}P~%DR1hN3~XRtvCm~Kxs-5#5C z8}$jnOABGoW)?huAT~HOm>7s9;l5S9gx6NXgam*%>b@;$Qx&TouoP@_V@?wSs`G*Q Mlirpdjt_49KS`z@Qvd(} literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/plan/window-aggregate-cumulate-event-time-with-offset.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/plan/window-aggregate-cumulate-event-time-with-offset.json new file mode 100644 index 00000000000..0212a971b37 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/plan/window-aggregate-cumulate-event-time-with-offset.json @@ -0,0 +1,499 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 151, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 152, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 153, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 154, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 155, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 156, + "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s], offset=[1 s])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 157, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 158, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 151, + "target" : 152, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 152, + "target" : 153, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 153, + "target" : 154, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 154, + "target" : 155, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 155, + "target" : 156, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 156, + "target" : 157, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 157, + "target" : 158, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time-with-offset/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..7a96552267adafd34d620929971591e714eb2efc GIT binary patch literal 23134 zcmeHPeT*B$6(9R9ms}FK3xsbHrR73th`;T-b5CmPyEuu#xwCBFrGyqUUax(dvv-}{ z^?jsjxCp7K6bTTtRZ-OvRR{qJDpEw%BK5lhiPF+9`bVn@C`h2TRF$AgKd938W_Rt~ z^E&Ze;v8Z=px%F4d$cKa6XzJ9x6r(#USOv#i4K@!bSYU{y`41ZZuVeH5%2rkdBLN%1AYt z6e<*l1Jn}^QXc-I1y8;Z;yutAJ{l>6!v4VswCgYM>uK3;>>F@pA{9@^vJ>fC zCL7DfbGs6mY&tPInN3WL=k`tPij)3KDw)VO65^~}|5PkF8P8?n=|n7YpPIIym_kdUMkCU!8^L3@+n!U2&yp4$1p0x z%K|M6hk1G6Y>63>R+VW<1-f1Evs@$Zp8U>x3y+I`dhMf$U+f><3W?a;Ma*OjDc+l| zUp=EOG)nzT859jkC>i5u40c8!OE+64To?svl z3WWHezu-q%`#$`btTo!Lcdi{7TQfNh^2Q3-3s!0wf|ZoSl1YfHk4weIRqK1{#d-dg$i%lYSYr(E~-+~de{`RWJz?L ze4DIMWa4i@1GjO-Tb4=!N6X$RDvwQ5ISY~cu({mCl^{K3UM#88^X=A?)atX=<32#v z3(9z{T*lrd$vUK+mLydrgGp$fPg6B~Ox5r)7Wi26e0uYIrfRX=$b8jxL?+?3y0)aO z-aT>!A<(PK57JrcEmN`Njf=I4IkBpewNtV5*xp$Bo-=38kj?x52g%wlexL>A=7Gzo zvLKe0jGbMiW%$;R?KzYHIj5j{>Z{5U$14idl$#U8f;8u0sR}g6(3)Jte#`xIQ&SaH z=4l1$Vp^S4w?)}{IkeRpoPH+t>}{vGlQ}mtnMRumw+4r;BbEW1qs*(4xbVcWC!hDu z#kC5X%{bc%t#vh4dFYbKEa@hl*>J~~(Q81`k~`sic8*2?zuPX|9c zCw^vUHM!2d@(_zc=k4O{EY`*y#Jo4 zQ~ko9FF8})&e_N6gfU=!v?!H$PPYB{B0Z z-`TQ-A>Gi<>kdFj2rqMZix#E*;{n-@b5-3t2F^`zqHq(O(2`!GxC@T`SVvyBbi^QK zWNYfk>yXl4Z&7nL+m0BHD@JytV(AE;z->8K)$8=)*mUE1=jE$^Ab$116H}i%a+Ub@ zM_z__tb7sEoBD#0gna(Q3u{05=J#If`u7VD{93)|A&6=!!aSVdNDz)CXFM{m%9K+f zRBC0FVbXe2c3@$+cD$s8w{FnGJ$iVr9{x}d|Eq;NhsZ{~>ZMigWc3rW^V@pXziOf9 zV|vKxAqpN`oIr$ z5S#*9a}&AWjtkau-25(C@9wk^*j) zlBkj^?=(TH6ZwNYrvkRxJp(QRD%oaZCnH@S(7hdUK+emndQ}F|>r6zGFs=E*UaCwZ zd8W*_cM<{hqouC*4gE-xq=&U#IYf>S*Tsk1uF7WLm7drS2GPyTVnX@#j!ES*3>gfi zGB7Dtq=HZsn1>9=^?QBW1JC^JzJA8N?lQy}yxV#13K|t(H%bM&7rg|69W+L z#u?oZjxG@V0UpoI6`^Hn(SVM)-C3j@?(y_dO; zV9;-M0Aj*>JrLZeoxP&vBC9|+*cJ!Sy9P>Bs@A#TPLSTJ%yR-_VD-5hwwb%g)g^9G z^*Y&np0~FBhHL^r)NE{DDz`t8-8(Uv&0$RLz{EKEes_T_yUdm&6@B2?K@!s21WZcD z_e>^Z={kef>kQ(>0*f1PVNTqeyYOa5J!2OqA_LC{Be3>ax1H+*)+)71PGCgZTf1yd zQLPk=)5SW8jfPE|!_Hvl0PAju4aapX=33}_W5%o$Va2-L0A!jl7j1r=_{XEeheuA0 zYM2XV+sc}-4gJgW2~0-VRCyGjshbZz_C)TCmjSZffQG{6bCos+ zF*Eqy>6vjFVvJ2P)`M#Z(zYG#Phra5v!p`IzT(d( z9zA{Z*f6Y$2?NW*lH;t*5My3cE056=?A91oD=^bZ2Z|mYlD{x3Te~adhs%}1?82$D zbD67N=ho{>b3|5WU+QV>OU|$}wO>C(d%t|C->+dmALZiGoS-Z5|FO}VE=}ec`k}t) zIMTO5?=#bBaZkrji|=k>!8LnVprXsHIE17c{u=`=;!Th^0(f0y^(b}b#PEaT2DKDB zb^uh=0vrIfFuLFXs2BQeJO@B!__zb0PA>r=&E_{lQ5*nOM^N!?1q7EI0M!9d;XMrp zK&^jkrlq_3vfqJs095_;eiNpheLxE`f)&lKExUUn(n#Xqc^ zu2FV5bd56W{;ttQ1R5`cXT~!`HMQ9&`+(~u#&09`zNRr_U^^4S?*HdOVmYn^S3 zF5>akI>G*R+ike%sLMFyf{(p&Gwfrv-m&^^O2+SlsRFNi+!ME(tUEyZ0kh$#To|Jw z$Ac4V;Q+yRs4;3cg>UG>bD!}nh=MY0j>**W;FgCmMm3q5QUzy0jOr;$@M&JS>8*+I z{Rn_X3gD;QXd8Ot*uK-Kcf;4n>N8H?unc&oF*djeI2;DrVM50Eb}tq`aqPG29{tSb z@45;2tFDJORw(;cjY=ZTu9}=A^FDr_=fKNX5`FcGcGrs;9<3^|^>*D1e%Wec5Msu9P8cmd9zp(Z9^yS?>xdu`fn$WuEX&+8qFjZcgx#$ri$t6Lx8 nZS^{#0l4@3+k}8>FL?fBCNrt{_^$s0pJ_}1 literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/plan/window-aggregate-cumulate-event-time.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/plan/window-aggregate-cumulate-event-time.json new file mode 100644 index 00000000000..84f9f6aa723 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/plan/window-aggregate-cumulate-event-time.json @@ -0,0 +1,498 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 121, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 122, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 123, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 124, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 125, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 126, + "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "CumulativeWindow", + "maxSize" : "PT15S", + "step" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 127, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 128, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 121, + "target" : 122, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 122, + "target" : 123, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 123, + "target" : 124, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 124, + "target" : 125, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 125, + "target" : 126, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 126, + "target" : 127, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 127, + "target" : 128, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-cumulate-event-time/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..d324c1df93bbb75147fa43db1a5271a1196795af GIT binary patch literal 23134 zcmeGk3yd4Zb!=ZQxqP?_q(B0pk+?t!@veV&wuvh0yEuu#xjWgu3j~2#uh+iK*}Kl} z`tA}{a1j!z6bTS*r6RQup@2#iL?xg_MO9UmLR(Nlty)o4Eu|o}f>y0)RjRf$eQ);T z_1V7jh1ev`j%1HLpZ8|oyqS6P=FLmoUGx${dfrAK7-M=!i+e z+-Br0V7^%hf7`-KpZ@2=H*NUgk06N+a2zN3eShGzGM-JxQ;AqIaWI~OR5zKDl_JAcIDT4S3MCjtM&oiNfl;cGCYA+8 zJ5mv^U-ER6(` zfQQxT#Zr2)lwK_D1#6ULleId#<}0%b+0xiN+U=A)2PDGaag!dvM*BK)WQ;2Vwtibj z_j8q9caVM~e?nG84H~YI%kHo>dOUYX;5BG<&x|PLHL|Tqa}h5KcyEOgQ1X(B0m?+W z*@iR;3?bzAaq2XppRd3$?riJ{Wf*-J4FWI#9|@*+jkxGI_R_Yihsiy}1taEb3y_VH zHDK<8WUZ)p!Qn@7mPt9N~P4F%)kHFCLaglon0;k~Q|Ii@Jw5u9K1$9}f;kyp5{ zd59}PdQ_PDKrUFT=5TuVLVt$ZqMN}JgAnh*8Ew!8n8O2%pBGeK5i2@VUuxs&w5Ets z5qVmINr4o&3V!2=b9-3iiHQ3PJZrO2AQ zq4qf0f=TD|qPAa^N$iiqHnD#qdmxe7H$IukVoLVl_!w>q9^m;Ewj8k-14lU?iL?=j zl#1`2OvX|cfz}!X5~Ms$8+2hw+?zf4V~17H!%Ij&wk-mVh{c4Y)l-iw>T~Z%VsBzB zV-gvFE>+>*6M2Xk3*ct4fpLRD$o3@#zTnSyT&6uS( z!bgTuK9xPg(0#Ed52+gYls*TsFzjbWVVs7T(j*(<;2MHW zY&zOEW6r)M@}YsYF-aT`taEC2!KwQps|5tv`&~3FRItaLQHH0W*+MKoAba4{T zln)K+Io!9>W3s`0HX*B*9Bf<2(gos0a#pj%I#KKHw&A0oOHXl&P_#(5VL@@xL%N4) zqM=V*MPzunT$Ux9W-rI70)yhVO)jnfYwLrDjI97F2-`NGb^q#McBZ9^b;phZ6{P@2 zftnvV;wVrZ1uC?K_h1wo~@LCjsjJ;S!|(_9c+b9$0hci;zlU9)W85_K2jHS2<{EDd0IM}x>DE5AH@ zyk^~%;5F-Pdr!xUY=yz3nfV`*8rp2u{rX2|U)Q(sbvjlio~7_Ern57?bg1CQ#tJRR z=sX@@tMPE$+p)11FR{@u&ljS3!5auhczDqi6}%i5DR}ewNW>Q^_=348 zJioeT8t6i$B{%7$jjZ{?$k$+)&`e9Q+~w89sjslBcx`q!u8!N?tRma98d%gQ_?AKw z*oq?af(noRidp>$ftTjuYd=#EbCKp-?&liDU)2Tjkmzt zf~f}Ud8`inrm+~j(4UBnHh5;~%;0V?pQ^`rKI=1CtBE}2*&myL>{ilL<5|1))PSj3 zn_+dmH$24HnEL$jNYnZe{(0|h=DH>Llc~LgGDaUGx>mNg#W;tqWe;68e(t;q;(YPo zbOFzobK!O3qrkIY+VlY8lyTk-3)*E$@Qf-zP0+l_iGQm@eB5ueUOg8 z7cjq}FPKQkn=b}~pMU$^$SXJe>iXZl^kx;3Xce#m%13fJblw@}1-}r2tL|_BOp5~H zyugR@k?;@{U2hbJhz1jhVH346Emsv@NC_Nx8BOB_r%|fQE=c|Uz;M*hhtV-<&>ISa zVN@g#^m+Y-U{Dy&ae03f+2K9-P@AZx?Z&>fV;i*+|D#I7QKj`cl0%1t%*YWoCOxldh^2aHEGw zy#*P#34ZidB~UrSEp{F}c-Gp6ile4`qtOkvR+Kf#fpu^fz~Ca_CC}du5z?l|TdaQa z_|Mmzx^2@7)RW^{Xk(e9PWm)5)ab;Jmlc5(<^>)!owCGQfc99`Oka;0Y3c`Z6L>7u zYJ(7?-ZM3_uMt(VUfbI}2wnghAOS7FDOs?A2-t4MYB$rQ-6lJ=;|a8HG&VLqmKcpC sL2GP`@HPu3Gyur4$T4pdR NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 79, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 80, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 81, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$MOD$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$HASH_CODE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1024, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, MOD(HASH_CODE(comment), 1024) AS $f5, rowtime])" + }, { + "id" : 82, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 3 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 4, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name, $f5], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s])], select=[name, $f5, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 83, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name, $f5]])" + }, { + "id" : 84, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 1 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 6, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` INT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name, $f5], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s])], select=[name, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 85, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" + }, { + "id" : 86, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : 1, + "windowEnd" : 2, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `count1$3` BIGINT, `$window_end` BIGINT>", + "description" : "LocalWindowAggregate(groupBy=[name], window=[HOP(win_start=[window_start], win_end=[window_end], size=[10 s], slide=[5 s])], select=[name, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, COUNT(*) AS count1$3, slice_end('w$) AS $window_end])" + }, { + "id" : 87, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `count1$3` BIGINT, `$window_end` BIGINT>", + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 88, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : -1, + "windowEnd" : 5, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f1` BIGINT NOT NULL, `$f2` INT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[HOP(win_end=[$window_end], size=[10 s], slide=[5 s])], select=[name, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, COUNT(count1$3) AS window_start, start('w$) AS window_end])" + }, { + "id" : 89, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f1 AS $f3, $f2 AS $f4, $f3 AS $f5])" + }, { + "id" : 90, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, $f3, $f4, $f5])" + } ], + "edges" : [ { + "source" : 78, + "target" : 79, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 79, + "target" : 80, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 80, + "target" : 81, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 81, + "target" : 82, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 82, + "target" : 83, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 83, + "target" : 84, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 84, + "target" : 85, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 85, + "target" : 86, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 86, + "target" : 87, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 87, + "target" : 88, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 88, + "target" : 89, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 89, + "target" : 90, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..544b1207727580c003cfd289eb6e8026bb31915b GIT binary patch literal 30037 zcmeGlZHybmb?m!bKH)AQZNqnHE|fyn-}c#!D)3#L#NgbWY~KY!i!p#@uToD@6}41OsDU8VH44E{~WNLT*1F z0#BZLg8?3*EzzTc!ycm$8-ivdOB02X%!_f63Zk4)sVeHRzC9*o#jiQXsq7ppWFb&- z=G@Lyz_Y^_O1WLTclrZ6#ymT{#e62y=fWqm)0uQ`FXbiG?db~y+#bI--a=t!+=cv~$aPF~yz^7dS zG3ZGHWbeYSt0CN2H^9o!a3mg14#iW6WH=c~t&1j-@o4`@GCDMv+BCE-LV6Ozv1qaq z5Mk+hM#Hg@NGcJDN5iq`mPj0Y?L;of83!#=ew*kR%S!nj@F6+)LM~UxJJel8v7|~_ z#gV3p=%BJpcRBiZsp7CK@cl>+S~K zy;XOAt-BwQC3==qvs%-qMOnl0CuGgz`uD#ym-~wDBE7@IJ!B(oCf@+{r2~T7&4p;t z!});zxsWRa)Jz4p81(u4UY8F5`x`(9!?bT@co}B4qx=seGjGV#1j8mnvnu0fiE(z;C*-9(FPZzSD%_;5ji5u}h;$$2-LfF4S3 z7kL$+?%W~e1(mF<11^^9gmSNg7!dQ^qMnse==r8XW6+I7VIx(xVeykiXvQ_QHDLs! zHKUFI2jq_k)9Yqq?byX&S9X#;#0D+qYAPcWCM%iQw~$4WGFZxHvFI3CfQ82k1yv=1 zIyJ^fjT;ElLDk3}nfPW35~kI`BFhMYTAhoe=|Bo{MDnGaIk2dbj`+}Kz~SXA64I|@ zppeZ%lR;A|y(9TtL6F8IRwR(}$O^ zY;3$rzG!OUN;Iu`&#UCjW+&`lllxx1`2H8^k2;7gN;(yossLRuR!v~{?t=RGo5Gt; z6TRTB#~!WY28g2r2wxBtUY3d)Q7<#mbV8M-d7i{pFeWD)Snf&h@_ zXlj(o1KX%Pg+QaED=X$R>b6PD_Z-_Ebb3@We>5B)*cguAa_ZD6M!Xk5flDT!_Y@{l zib`b_3idRGfh3HmkRn&CF4`U?3yZSIOE?sj`>wA!N-pQg`tuanFHA-^MS#)SG@RNT zO>P_-Nv6;zwPk1!*_sZZ`58ivrRWXEC?1it6p$2;Y#51!<7EUb(h(%)1?Jal!|LLtqKU8gN!K4c%+ z5UEsli2%Js%JXUpD}(iOv+8;oXy-GZ6;Sm`kC}cl46B8l#2b^=1qTve{on3Y9y<&$ zn719~lb*^jqxFx6-#-}Kz4pm|j8tR=Ucx|FSti@7A=#H$GB&Zt3NWcb5h4=M=;hlV zJ(N1-VDn<90KO{ug4PEyGR$XsWbB6MQzsbn!7&6utlQdGW5~v7!T~_j@EI#WufOB% zdtSbK)!$+0x|Ylvr`eJSW2~w2gQZ+rly5(L-^Gi+`{o^$%Io-Ng<-{)R`3PQD`rG_ z*;@}E{BYm?AY>givPJ+0S(-kEypWb2y@%MV(X50l-~2H&sK${c8_;eo4+TRXwyKGR zs`07fpnzm_?&;GR*_s_8+cxXk+A-^zM{pgTGaU7*N1RV{yq=pa(w=##t1>TH&Cb|< zojo_NPjvRTEgkW*YWCSM7jv_N>P(d0Z9XyIG&hlF@P|6~Eb5!-GFhiS6O+|NwlvLT zX$HLw=WvjR;MKtg?m4fVI2WekP!!>nkES^So+r*bvf12vhK?Y}# zD=^!pOsl@C@@x&HQ)R}M02R3aOMoi$@3I7_mH^c;tpv6r8MOqcY#T^%7}9-9fNBX) z_5E&bw@}4-rA!`{0M!zpS^`vK(-?Sud|;ca72B3=k=s}TRMhP`-A?~pkfd31bEXVp z2~g|J`}KEc=cWwM5}=y)5N9g?vjnKH-R_tL0xbcmrg+;xB-`7--z1XMM76CEOP)Of zRAgL(_=gYGoPCoAUg<6YehxgYxu^w(&mzqM1=jLYla@T7rRjNIb3xygJ)k_t9+PTw zIc-`C+BY?=QO-g=t?>yJNfJ}pJS2(j8o!^4@>Mmj|3tbXVC_xg_o2TmT(+NW zWqp192*$z*kR`}DD(}D;2Duz*yAC@XCJBjNmzSeK9*$s8zrdxvK9TdKc}jy`ml$-7 z0f0y0$BtlB;MT!z=IF(|kwG|y!4lLzZ_%3Z7z-?QAG_$E@M4@$&>90htY+xi$|B7{UhMd3p(x5!Ey#)k zm$`C9|Hibu?I7CWJUDDvR%ExvlUW=e`>gh1tnp-Sjwh{I39YOeqr`DBuUf-Xo#U+* zyHoFUtHti%p8ss*_MaPer!~XVVx3i@nT&4j0VHBQ*F|oKtkcbTBySjwka^5j2XIn$ zQlEor<+`cU()x!^fCuJgy6b0cna*Oz&lou@>T4tQ#px6)7hqLn9-pPVL8M>LmdGNd zZ0o1)QUgwBI5p!z%v=TA!*pU-s;uGA)=6_B851$-g1fU>1+60<-VEos8=afSb1iUe zosLW5gad4qjDoAz8Xac2#8I(8CK_2X-KG&Y3bfcS~KCcLK4y%NN=h^r`EW2&fwFD zQ8Xkl<@6Tx{qn&u80#1g57d|$YYocIF|1aEc|PSb zNUIJ%)v+l&48e`KsakndSw7D#Lsb8NBunRxxGxCtoLd7SW({1)jNB| zL6x)KXV^Dtqpp4q{+fyIgLnqFURDUy?dr1J@DBmGqWlFfsBqeJ5exK`wo|8APTU(+ zRnkM#n!E1=g3q3*ReA$_SLk76n&@?_Wav7Y-Kr|#KDnpaA{GvSQD)fATIGdQhSo2^ z3L4MxMIY@Iu08nvYu`Hf(V91(eFXfm=)D+jz<}?jXWgr@AFls^yZMoc4cl$p+i1YI zCdnD5#~JjB)a`M3I9>>VEn=GXb3y9nxscyC*5~)TXvl-DVd-(2n%R1_FkoTeY{fv^ z^*B9(j|ao2yKHNr_IviWJh*qwJmQl z0bR8~ZM8tBW3ForTGXrEByd(zZ)>(K=B|#K)fueL0Q%hKi-0VRD7DJ;)aOg-!P1B_ zwdYLL$t;a1kdV)YZ>Xj|%Zl+#3DA&pS5DYiYC5f93ERD+XNW?SN1r(%zEDV9KlGd1 zj^=)bH__F|e~q}hd-Y{kU3!J{&K0k3x#_`cy6?TTi6l?_e(U3}Y<&9UW3S)hTk+c0 zQTBhjQuH-{utxdAP=|sG3;7-F!bQhG`E^*vqhqQh8p_pnJ6W)W^uYclPi0|%=6Mlr zs}i;lR2@RE^%U-bgzs+STgpqyHnUHnl!p6v(8tIoQBsOZU%gKzZt5j0=TC3%70h?NBgfP1d zOeGs@i>g;^Gu*vk)VL-mzy;i~;;lgf!tE%-?WhAcLm~V%bFtaNf$-qaV00iH1I<^x pg;$pI1ONaXQ*O$vLyF}D*a~%PV}=Q@ NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 109, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 110, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 111, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$MOD$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$HASH_CODE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1024, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, MOD(HASH_CODE(comment), 1024) AS $f5, rowtime])" + }, { + "id" : 112, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 3 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 4, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name, $f5], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, $f5, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 113, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name, $f5]])" + }, { + "id" : 114, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 1 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 6, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` INT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name, $f5], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 115, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" + }, { + "id" : 116, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : 1, + "windowEnd" : 2, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `count1$3` BIGINT, `$window_end` BIGINT>", + "description" : "LocalWindowAggregate(groupBy=[name], window=[HOP(win_start=[window_start], win_end=[window_end], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, COUNT(*) AS count1$3, slice_end('w$) AS $window_end])" + }, { + "id" : 117, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `count1$3` BIGINT, `$window_end` BIGINT>", + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 118, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : -1, + "windowEnd" : 5, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f1` BIGINT NOT NULL, `$f2` INT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[HOP(win_end=[$window_end], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, COUNT(count1$3) AS window_start, start('w$) AS window_end])" + }, { + "id" : 119, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f1 AS $f3, $f2 AS $f4, $f3 AS $f5])" + }, { + "id" : 120, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, $f3, $f4, $f5])" + } ], + "edges" : [ { + "source" : 108, + "target" : 109, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 109, + "target" : 110, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 110, + "target" : 111, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 111, + "target" : 112, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 112, + "target" : 113, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 113, + "target" : 114, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 114, + "target" : 115, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 115, + "target" : 116, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 116, + "target" : 117, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 117, + "target" : 118, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 118, + "target" : 119, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 119, + "target" : 120, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..58b37a40a030007e3aded766f83a8485c0d2366e GIT binary patch literal 30031 zcmeGlTZ|jk_1fNSvU%+WQW^q8X_L?t;`h6eO1&E=F*v)H?cIA}Z}i)k?h7Qr`3tMAcBG=iHgG$LsON z%VLwTGm?F0?(>{;&$)Bv+;i`Z?H;ugLb~9GmqiK4q9bQd{bOJ594)tgMpAgkn@N9-movrK-R^&s<1|_S_`&gA&OiN@ipZGeN@F^L&B(jB|_;o%vD$5+!HR?aTx``+VVy+jVfiKd>+F+3zh2qoY9= zUZeY+Y3JUGz{_rTz~u`EeIc*U<#LBZZokVZF=Jf0#0#=LpAWh7oZXXmF=4+e5Dw&7 zXUGq?K{jmX!ak4P7YMTUaKM$b2f}_&mh&@g(Cr4WUV~qEQ?L;=Aj(i8nvA4}l9^OG zl8$CJ#Zu{HtbaHi8yd`P8`=~leW^q|maZj4X}P|UNPIY&Nkx;fNIbSPnuJ&@5liBz zgDErY9?p?3@WMWL;2mtKSS$$+`9PVg$b3O^WEqKbFrvsDaP%LLxrE3Ud6^&MA{dq8 zM4l<|2RU)=bcrc}DNB1wGPG-hU)^=vI~&ufcb5w1uKCtio_wR{aY#gLCoz-Oqj;~o z-g@!CuJpM#mtA+`(N|wMd+Qh^U8u+t&%gaf;nqrQF8RgTrA9?4cUsAE2`FwCIay>_8A7R2kR`}Z z=HnkB*WIUvXO!^zMQXT34ewCHU#sCcvRJKhYE>J8N|p_@enK|U zfpdrzLFS8`PEw*1GA9a5K_?WAQFWrQt{WX_RM~0)izIPaN8DD#MDC4l7Pd zFDJ#qspfUYYh^Z0l*dG}vJpIrcCs|Ca3!%>Nf)$6aqT}a$P}TqULB&_nDWM3NUxfn zC`m9(l?0hwev1z1q3m9cl>zFmeY}v9$!Y^|v0W#$dlTeKcuM6mz+) zjI;-vpD05&ZfL}WGK|uVDghi&KN3vE%|iRJo5C*bC$|#|^q8xqigc8$Rc75u7V^?y zrBJ}8<75Fgo-CDQnFI_l#!SsS2+IL#c5dv5(Mb>m62RWibrKnFV z%fyx(+72|lmS#f!bqtgW1?V#9N@?w|P%P#6JWr7VzJ9NFo%_@qclS~1brm2+ququJ zS5T|?%6>BW+gn*1IEvB5-aj$g{d#o@pMWSd0`ZJS^%Ly3S0&E z*yDKLHBkn-(j)++HJ%z_#DP7GIE6uDq`SZgqw<~!tapj!b`1JtGJhnJ9M~F3e)0VI z^OSj4LW7IPL3&E#8A)bD85;JqM1eGns8FI_(8&CFQrZR1C6$N2+m2Gc;Vjri)=!-pG zf|nXJ!65^UT)X#s4`kQhv===t%NM!{y*qYMI%c$ z;oU|Z3Wh%HRhSw=oT<=st(_jVto96#4P`Z0C;zOptL2@ptcLH@vd9{EX|QU~ z5Y^DGhUbE}lyuerbmyyA3NW=vrGUqIdC6<$5LE~sfStntqsknl%_Ell51AVSRA|+Ym zm?G3gSGbnSoXE%}QF7o%u2}QF(S6+7h_W;fmKs*w*v;`|7K_I|tXUXyJeiy0Nqato zR@aSD+!!aw=I~^&xYbm5jMlcA>JHBM&&F#1kx_TrGdwL)RV8M0RNd+U>|uR3MR!Cu zsXK>!VX38EH zu&2m8w!j=bXuOliLaDl)km08Wtj%yL`xDX4D(DQR6Gf@^4Tn3Ov}TggF_UVyyObT! zY{|%WSjF9J)jY0hfko@g7|)G6V5gE12vw{xp!=3M8orwqOIEcUZYNTs^r^(7H=blq zzz#IHt4DuB;pPiz>U%5&-{l2PE(r%7KK#R{9OF@?LSJhY*4I_*NI*&V>E1=!^TKV1 zC8Rx&?rI>X(zymJ@R{)u+(ux^+AWY}gJ=VjN&U+IXp?rEFxA?69;>T<`CtUL>PSQe z8g?>k56PBjdsakPHsjJ*%Rry9ZHpuzxtTWAXpd?x8X%fA39G|i-6~4Q)K+q_Y)#dC zH;#*b+o@}(;ZH|xNu=8Q7|^k@wO5S!8GND7;QGO$-78p4c~OJF%Gs!MCbKLlib^$Bj! zWwhlc7N{jH=g-rc?9bJqq^4$-@X*@`KE0-*^c=*lP}9gX*$q+BJRS969ZEP)?rF7& zo&nhR8K$$=c%hW$^Ggsx?HYUHp=Wl#oL=?dt83m#5BPx^{@Z^nJ+!^ z_IHhnTwK1Qu0YVAb1`l^8}fqxQqIHK!}*-o9t>pJpeG*=y21=#Vfu;!5zNb^fJuQ% zodTWq74@)QcP^Cm*n?ii4?g(dk{|MOVS6qBM=!8}oR0~@2;2Jz(6s};q88Ivlr9RM z)zrUklj*vT;C360Y;)Z+(d1s`tb$9Dd)qTLGI#ypOkpsE0er^w=NXw^S90CvRv#_L z3e)RK*PkPfG}z^1BXV! zbFcQy3%s;PACszN;UFE1(W*&Rq%s`ugHhf25}ZW|H$4>rJRo7=Sq@GXtHFkTY=*P) zDjLpOR;Q+YU>Q7@3oi?S(|g>l5Fw?VR*A)r9sa|D<2Nt=B|TwkA;4H<=;1*!@yDUE z`g7dalE^u^362HF%aY)%R0CWhxzEr*lTZ+1l8Y5!QOR*TU|^ z4K)ER;II^L0}&8zM-^^|0o)ohwQ@pP&p>2wXfQSqiGvfX+QX}>bpilDjw$DA8n9xu Z0QQ0*HhP%we4QRRt0Og>N<;@Y{U5{ql`#MS literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/plan/window-aggregate-hop-event-time-two-phase-with-offset.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/plan/window-aggregate-hop-event-time-two-phase-with-offset.json new file mode 100644 index 00000000000..105398b0eeb --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/plan/window-aggregate-hop-event-time-two-phase-with-offset.json @@ -0,0 +1,649 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 99, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 100, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 101, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 102, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 103, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 104, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 105, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 5, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 106, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 107, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 99, + "target" : 100, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 100, + "target" : 101, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 101, + "target" : 102, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 102, + "target" : 103, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 103, + "target" : 104, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 104, + "target" : 105, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 105, + "target" : 106, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 106, + "target" : 107, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase-with-offset/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..58936287d7531e4833a065f549c7812fb89d94be GIT binary patch literal 23049 zcmeGkZHybmb?m!b?vlV=j-z}gjhYLj5PyF+NU87QBnId1YWpq_RLtIb?Ax5Zc6QhI zk+x}~NDU>ZAPN=yXptmossnfKnzynXZLMeokq2qB&DhmS=~NcYE2y|D3>oj?26`>#Fs_`Ut> zyR@=HSgu#Xcf*r=;mE^(n*K$?_v6D~JF~6-St!Dqhp>v?#Da*dWIz_gX`4h9iQ^SW zHM!Xz35M^gm>c267>&EdjO476zY?E#KCDzpbXW z8?}_z(&HBLU`EK%PdR2d$&o4K(o~Wid6y&QcTammDVOuWexH9j`U3%9Fysq59WpmViv>YcY`%cU?WZ>C_WCjbjymb2#}V}U z;4Q=3+^zs;^Lm^a+oa3ywgo(XUnn@~4pDcI5ZkYygTRNz)I%a$$cE9eNIaYzji(aH za59qG9!(_U(V_8VbaW)Odvtq*^d-h((PX6{!rJvsgk$58R3Z|OhGWsakvQa9$wWcQ z+PNafPf>d&2VAo&++>d0rIM%!d1_bYi?pN&IoTeWQ>Y|zxdfF2E+-tI(qL^?Lgb3_ zR6zmQ9pth%Q5q~K|3X&C61lknCd44ycL+3>mbY^XmxSN{_|D1ePaQsfV`CkfAK~(Vt=BNn-CS|&oupSQA1la$0v%V#6?YmsJvzCM@(T32 zb6OD73fWMnyNK5TcyEImQ1kMl21-YIjR9#41cIh_aq<+RpDO|zx73b=35?1{jR1Cl zj|5YPMl6hvU99i&esYjlfMd?402wG*0cPDxRtfS*DVIakG1865dBd|e%T>rOslk4eszS762*O`MHopxK622cTf%@8^C!d>sLXJ9P6LOy2T zaHX2c-YY7Q!;-|!ZkkLCLb?NI zv_=~cjt&s~G?jTtD5^r;Z;ZqyS=f zQxr(TiV7|A#cI^nDCsFmloxO*D$iYSbCO(!$>#HdzF(J&?vB7Vv3o4FCz{+fI-X2n zPHOMy2yO};AoI)YIbzWm$0Qz!v=)dIkL(m$ zj5-rQk{cX8TK=+GEPIxsZ)$_a1FsO*5&p#EZMb4Idssle8vhe>yN$hz>D{9{4Fe9 zw~`ezT&_f6jWrB@q?Dhe(w9y?yl(YFzdKeLyheY93Q%2e=X1TjG&$0qeW|OmFPUs-YQLtQ8^2F9^|vh( z@uOn5o`J737>3!=fYH_6G=j_S5SFRlJ*7()jc zSq4;4whch5epSJ2O-~ovj7?@ym=Hb7;=zHYU z>K`@HB=+b-zX`4AHjV=~e^1Egq@3G384Tbxgg?U{tV#{~jbl6C?)%a* z&f+-q=d};bD|Ac}^58x`Lz#=O;ec+jLc`{$uY8Od_;2LdcV-@>+cw{R;+YTTZ#;c3 z8|eap)>zsz3tW`)oHVV{Lxu5l+b?DKT~04`dz>~WPlGmZ2>2TEx_Mj3M+3eL@8?54 zH}viM@W)tM>9;nxW>~pmd<1S~um<(7U$jQSK!l~gs|St$D70qcw0% zs3ry4L4kFr+JWVx#L@mG)<-8@oGWMxQoqOM4e%jb$nA97Jd@NL;?sP9b8&>Mp8}#V zUEWGs=}E2MLvz+_ne}X0OC51unpQ32L4d(K%hGc1tQBNaN&&ws32)pe)KQcQJeA=} zXf~x@G_kQ#yaTE$Cl*qF%l&i>|7As@TpnP-MM%M3C{l@23X*I`D}hwej;a(O^=>2F z)hyowFs1iH+`eo3F_wXGt(V@`Xn7l|W(1v~qGC>lI(JdcrM%u%)Q@Z_Z_Q*_t!)Eb zHmd~!fN$--+ah;Fwrls~lXr|o$O?7|9(D<9gWaK(8c3(<5UyRMKgaV7^Ev`vL|ZQL z%+s{5Fp<0w6%Zi4}Z~J8++wL=(-2}re3W5R-3*lWHC)NjIa zJB&3^q4d*Z&)j;NKMCPUFyzBfP2q0eYJQSPiSf zNEnhqyX_>c*>EqOL(IEs$f+{7&h>0=b^`C1FSytPbGc4-ozKDQYvGXwcpC`SV6}`@ zfgczNLmY>(@NiB1p4Kq;GCiO{nCDYYU9{@xQysg*V^G{ko2vcRZag($YF1`g)#$Z2 zLI$C}aN-s<4%KkM$0WL85&5)N?#pT1$Aqnw&22H}N7s@cT?Tn>y$Rw%iSu<4&zJKN zo8-d{=jS6A(`DG{_x)xfPSy60rVFsHg^WjOb6edbiJ z_C0M^e7yhTsgB+*n~oh~jv1Se!;d~~juYKCQgzzce1hepW8}JH@NcndUMSbKbLI|) zO%V}3#&olqrT)hQvYVCy-7^eP0^mepqX%m1-kHR5_AGGi6C=%~tA(EG$Vkk_H4 zuiT^BYIO&)vRqGnK`d1RK}X1SOM&jzpkUYaN3-MvYZPGplMtez^3yD!t9`iqIws-K zpdwJ+z-+aW?!BZB)^MJa(!-p{QwZjm-b?WF3A1)^5N8&C_v*g|QIMyMIf>FFgr~zC z9ZaGm7a?dKW_4u>;QR(JJtYxLM{pXMhIpNozM%o2?i!~J!%?Z~&_AfK4bEnKECgcW zxtbwDLXU5Y%}*TtdG}+t4}6!!ZCnL?%yTRxpF(`KAq)9}L>+XF@?fzlh>kL#9pz?v zWJiV6jruqM@kx~m6T&Q3rb2erqN+D)W4{OC&VLOgpap~=^VARl`)x1x+g_*Nx-c~p z_p@g>JTf{G9S+C9`loStLm4J?0LZZ*3~wD#EH^+d)E$knPZ+hX2O?l4#uH NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 70, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 71, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 72, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 73, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s])], select=[name, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 74, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 75, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "HoppingWindow", + "size" : "PT10S", + "slide" : "PT5S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 5, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s])], select=[name, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 76, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 77, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 69, + "target" : 70, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 70, + "target" : 71, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 71, + "target" : 72, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 72, + "target" : 73, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 73, + "target" : 74, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 74, + "target" : 75, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 75, + "target" : 76, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 76, + "target" : 77, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-two-phase/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..167c42668156f536ea917781f3e7f8d52aabc015 GIT binary patch literal 23045 zcmeHPeQX=$8Gr4hY10;xer$Y}MI!}P>d)ATO{6g=bz6g*WVVw+kT9>$XXhIG&b>P) zY0w~DHHs)&$3Cw)lY8EsT6pS2$Ku0Z zd{s|d!gM1O-kYumTz3pz|HiSqf4O*<|DDsn9)={$xrAAo6uJJ-r{$-MU;g&7X_qV2 z7fI+jmt3STIF$F({7{e<28RatXfQGmrv89G5Do_Y1CdY^vTj~hu|dRzt#OepX{jo6 ze2S-eUQVl22X#A0#C zk5}=qFXKB@6$LdAj)X)0!NF+6?+=IlQU74jr_edRA_=1E2^MH5L_Pi>Kg7{Oln(_0 zzAzUI1_r~_6V4CiJRyFt;K^};f+v^@`=KpEImiiYy$!#vrfg%=Kq%uA@l-4`p30^( zu}nO>Cy~yi5+jqD#Q0eD!1$gx=}k{06PbEKoR#aHiX|uG*>pUWh$R!#@f5_mi7d%Q zFRf5+hW8dqf;bBgf|rxZWl8j^M=E?(6-tUXM-|>nWtkrFjvP_>30Wu$sxZgLFe=T< z0xb!PyxjkBiD{8mlo?3{x*hPdT%ki>dvX2OAKdl7uS$2`wC%kIA+cfD=sns!51lBN zsXVL6L#yZ6weHaY7?Op&KjaC}AoYX>auE*=4^mGw8sTiA7UaSs#~oj;q@9^40$U%49wM!L`Kx+xB<$n{BJwso0S~ z+a<8$blXXM!UWcOg^dKF{(NXC;E6=Jh$j>dgglX&F`TqH7$n?G0Rp&*Qs~NIz=Y_CNyxbRJI`Dj7@y zdp>PH_p$xl$3);`?D=%|e5Pu#-0(uhb&pI!Ej4XfS-p8=DMc>R=#7ij zvN^G$lJ!%u)ad?L>fSSF&XDZ~KZ0a!51(#Axv}HYsw{}bWqoHCY3aUoWM>vffSgrO z1@u*9iQ^RoYRb+FVqTi}uv7&YGE@^6vEOmO(!i>s$~-MYT}*Y7D!fcpNmjfWBv>C| z7KSc)3)NG%NeIs9bw+QK!O|Wjug@|g_&Huwoyowq3$|7|OZkPll(%OxY|^&@S}fuw zfER24x5jUe@6pZd%C`n4U&9WhX*t$mJ0cVlqMETNAh<)f zroWgrc_s`HhnswA^^UHQ#JMwj32mgMIM4>dYe+CXygeeskt`arSA`Ggv?J}Fw# zdRH_0Qgq%rb=^s_jw{i{MU`wx;>zqpCEGVzqKYz{Bpa0)24|~A0jTHGKs{y&Fi_8j z6zVF+@I|^fbfjVj=VWqYlT`uX&^(xma{@o_JwPi{5E{J?;X(y$LT{6@wQ^{y)g}Gx z#BXkShC7vYGm~kgp>T5q*4krfusKS+Dv3v)JpR;+-ubvzVWSymTcNeC`YI1iGMOcv zq&*w%*a}(=OL1Lz{gx+-QPmz!?`+_-DX2i_r)a6lFSAZzF55&~IM>WUVv@Uy#GBA< zGN}5kW6V+5rEzM`V}e1w|D^PK|L~)^7|g8-1$dKx_h%n`u*xwzlm^ zW``3nHS2@K(t1s}E|YNGWU7g98v%#-0-QF8)HHqj8>8&#>HB4~bw!26SMkPFsVs0f zU~oEKd4Kv#|5>~?sLkV@O?cEw8f9KM`Tq7FxsR1WJxh`vP7>$fC`*t;wlmmo7MpY! z2uZ3WxMIi%;38NsuWP~{j!$nLkS)W!zbSS)}KyZ`*(naZz1dEaNht0tha7-k5$V#UcT?t_jYXh?yDzoNkCQOGiCgd zwE&%U_1n6+wrAg4>wWfe%RZYGLCe$tZhg^R6@=kTb!V!d|MMkhs@r+gSbZ@1t&bL^ z63@wYo>(I2m(l+#c<9yVILT=@M**V4wCC6(Y!_t53HbZCY#~TDv~zY4heeSPUgq!y zElT^x1F~IbtGas>Jc!^#;o1Sn-E9mw)P zr#B^*F5C9{yWfBNwRa)vY(*ZK`0(L(e)`b!m2aGUe8hjRtxFWEyZ-mqGdKL=@sZae zlhf}VAAUXh2RkaviwX`N;qY_TBlD_EITb>sT2dMEt&45fceHAEy{v^dZqmbjdU#9^ zf3JuCC7bmsmshoWNXxQYN7i>gtv~-$3q9BCA*Y9ruC=E@C=wpv!uh;skkXJR6dlZY za(p502?qm_NIn`Z&>?U&^!yF@15bnI!ox0Y#jR^44tf{LcDz^MOL@hxI{*FiQ&*op z`Y1qHt|Ym%fN4gOqLN+>p+uC?mjoyPp~VZ_n!U2onq>w$xP@q=l61I9XOe7UZ5=r> zM$2G3_m~XNf+Z?IZW;$t_u$gUK0(X zBQ<^ytgP#p8HVH6J1Uh*&}Gn-O8=x-mhwVDU~V@su0QPA8F=nuTxA-mmY zub^J>6(eNnk9U4>4Xeh*spRvf9X6Ag0^K0SgUv@nFBK=_px)^mT;E4+u zF057axW#ord%}&Q+vcQO9~1o$?!_5h9~>Pp@bkRF$wEaN)LTtsTElv9Op7onPy$!M zJ)%^2-?>nMLnM|2jGQ0z2kX7Wbq@x;R<9y9c&`M4o3#U7G>2pr2nSn|0J`iziE_m{ zH{1!*U6FZCKwzylcSAOF7p1z)*Q!=0o6ieY-ml6e0GZ9i4oqYZCNlfSCo@@$$xe@t zq1Sc?7|1K+9I5CX$4;k^-WCv2D!y+r8B5g!v{4rjFXmZXzYBBXR@{L%yJ{IbI1vSS zF2IAe&ALHeBd}hnma_sQ(%RZ(b2MwEU>sT2NNhF?+ z7Lw8BUu*al)^O%H2;IvUd3o#Wk3I3h(c=+V6%z(Ph9$>XnIR^;s8$}M$JwpXtyZ+B zon9GTI?jK7R<>4G$j_F4hS@Jv7v~CBz4oow7iW*G&A!xC-&v*Hkvsyl`B zH;Fgl>khP=Ko@2;Qk8jm-$k+i;Q44SY)cnX12S7dFulP9tD#OPe0Ci>=2x&II z8H(Zns5*j*uRtKU8R4?TO~pW)(c0~Vj356}1zvY_ zCvG>{Fim;^JK?CDAEhG4gEMJ$9NV*L76efq^mh_kHZ+FnsimE zfYTmEbrmG|wk_OrS4H?Z1i%w{@E5N0hF&qYFD&ZK@YS>Wj8iZy172cG7%l>ie}Ptn zkT$-4i^U&1{>p|YZrlExn}FZwM&Pkb*@tLU5^i*v2~msR_mP*W4^j7+22UYuGMC~`{B%?p(c<8IAIPn2m$%_*7)|?@NJAI1Q9Kc zb2i#&_h@Wvd@L~I4Qr9Lv7bYcmvU1+W)vTVv)4fz594z)4T0C*os! F{trzTI=TP= literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/plan/window-aggregate-hop-event-time-with-offset.json similarity index 67% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/plan/window-aggregate-hop-event-time-with-offset.json index fd31d4ac513..5109b9f7f63 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/plan/window-aggregate-hop-event-time-with-offset.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { - "id" : 1, + "id" : 91, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" }, { - "name" : "c", + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,18 +85,24 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { - "id" : 2, + "id" : 92, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,10 +134,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { - "id" : 3, + "id" : 93, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "CALL", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -166,13 +184,9 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { - "id" : 4, + "id" : 94, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "VARCHAR(2147483647)" @@ -180,6 +194,10 @@ "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 3, @@ -200,14 +218,14 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -217,62 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, c, a, rowtime])" - }, { - "id" : 5, - "type" : "stream-exec-local-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$1", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$2", - "internalName" : "$SUM$1", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "HoppingWindow", - "size" : "PT10S", - "slide" : "PT5S", - "offset" : "PT5S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 3, - "isRowtime" : true - }, - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count1$2` BIGINT, `$slice_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[b], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s], offset=[5 s])], select=[b, COUNT(c) AS count$0, SUM(a) AS sum$1, COUNT(*) AS count1$2, slice_end('w$) AS $slice_end])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { - "id" : 6, + "id" : 95, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -282,49 +247,78 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count1$2` BIGINT, `$slice_end` BIGINT>", - "description" : "Exchange(distribution=[hash[b]])" + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, - "type" : "stream-exec-global-window-aggregate_1", + "id" : 96, + "type" : "stream-exec-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" }, "grouping" : [ 0 ], "aggCalls" : [ { - "name" : "EXPR$1", + "name" : "EXPR$3", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" }, { - "name" : "EXPR$2", + "name" : "EXPR$4", "internalName" : "$SUM$1", - "argList" : [ 2 ], + "argList" : [ 1 ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], "windowing" : { - "strategy" : "SliceAttached", + "strategy" : "TimeAttribute", "window" : { "type" : "HoppingWindow", "size" : "PT10S", "slide" : "PT5S", - "offset" : "PT5S" + "offset" : "PT1S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 4, + "timeAttributeIndex" : 3, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -354,7 +348,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -362,35 +355,23 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "localAggInputRowType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s], offset=[5 s])], select=[b, COUNT(count$0) AS EXPR$1, SUM(sum$1) AS EXPR$2, COUNT(count1$2) AS window_start, start('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s], offset=[1 s])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 97, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", "inputIndex" : 1, @@ -399,6 +380,10 @@ "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" } ], "condition" : null, "inputProperties" : [ { @@ -408,10 +393,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>", - "description" : "Calc(select=[b, EXPR$1, EXPR$2])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 98, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -422,25 +407,31 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" }, { "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -452,61 +443,54 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1, EXPR$2])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, + "source" : 91, + "target" : 92, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 92, + "target" : 93, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 93, + "target" : 94, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 94, + "target" : 95, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 95, + "target" : 96, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 96, + "target" : 97, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 97, + "target" : 98, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time-with-offset/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..59d0ed23704f395d9eb41b8e27d9cb05035bb9cf GIT binary patch literal 22998 zcmeGkZHybmb?o~{?hvlh|q%CqEc0DqBg2T6-3oERn-cW9~HEs0tux4pi)}_5|kE%L?!yE^u5_# zdwpKt`ABS%W>>l!@9fNbZ{EC_dGlu8OWt4b5JI})4@ktJkTe%rpOGtyK+}}ZQzfJE8Z}CNeMT(O&vJ9T z!p+D<_^5IvKbH#!4u@hnzwg+Q;qc*^z>#2CDik6)i+bY7V8gaR?%6Q%y3 zCl(I-ha=NI8uLYgvUlOHw<+D&HsH$mL?Ru}j;C{(Y&@ID?M`O0>Ey^{HaR|)+dsZL zK?XAusbseHA;IbmPQ_D`iCiX;PR3KogNZccx``qy1utLbg<0yIDT>lzcoDsVTq?%yrs)&41JVuqFr9NgPzO2s58sP1M zzq;qC=fC>TH-=yG3yRd)-EsD{w|Bko+w_wb zs@%8!JLT?Q{_IhAbjyb~bDKZcLX~wNUH|SMx4bINym4pWjSu|x2Xi*#DzLqR?qi&2jd4uix8g|H_^1JvX5N27s!Y&hs24ie(|0}u!j z-<*5x5c;VGq1X87BK0a2NfS%dBKL^}jVcmfw1{|PR-Is9_coh(&b>!^RE6Bp1QU)d z)w@Te#k{(k*Z3^F|NY13Zh7Y9>71LmQe@o>eltI*8W~I-;^%k{ha;B~Rc+UTLbfjn z-K_k=;~#GPp8LU)o48YCK#-+5nC(Pa8ZGiF0A8;HF7i@=V_CaWWKB^nhyq_kvNRG% z0Ul1DBsexnaExCZYXxVNh8m$l-I~s8*&jZ2YBy>5>Rqo*#KoC zy~%q4n@<8 z668%tm6G*iStDKP@dLoaTUjBLU+<_~EJBw-SE@skQc2E>Ga_pg$n}2)w)-#q?Q4UK zd)-CIF?qLhc>p!QH;s@JFWvXi76!&8XygV<57&z6-FtlvGF#Aad@Y=Ob@R7hg1KrQhj%ZuXSgN488R^h=^h->25&$(x9WHyI6xr5_ls0np}%&(Ag#A0+DlXzs(dSFsI zv3D{RPgfb#Z!m~Td6qZi!jiZacj1rQs|C9R2^q+>#J~};kd(A~?vX`p?j1?)O^#)& z2v(_;QVx_+K+@XWwJe5J06wudQETklJd6@i60{sP2HWRn%?&crZelq%Lepy_X6eN+ zuE`})Fc+;&k7PdcuVdQ+ZkS*Ywj0)y!P+#VkB>7SoQ)pab$$d3l^Mb1z+(j>27rXP2tC3IXV-O3&dS(>HVTd_4pAinuAxNTa zZ{LYu_AT)p2((O}DH%rnskff^-Q(N-1XI^tq-TyVRw&G|md1}&O4C$%=-iW=*L?HO zr)rHi_|K4G!`IjN1>H*vRN3&xnX@0Ad^ifAlO|{+aE#T7Fy_Vd`j|b=ewy7%$i@Tz zg$~scd{Os89_#bs7|ijG;YUo|&!H8M0%muB#KPu5|`C*tx=4uYSh4 zob&ay*(04?KjkO>;A-0e>*Y|KdElN6y@UD9H530Wel5-tv6kp%ro^v zgL{tjt#q4gNS{T?>LmwTRNTzVjsev%pc+Ty`e~tt>q=ET90RIjKy?hL=Akj@{P@5zS38a^J7TwS z45)Y=Xt|UAYr&G{*v;8x5XXROx9;1Y&R&}~K*xY;IYV5j{m(I=!g0Iz5*X+hQ1!dI zO-!;wP4X=!N!?W29<$`tV?ag46)$76hu4#n_doXXKn3_yKCh>E3k;urJ%J0X)u&o_ zTF*N9&S|~QKVHyNvuxiIbxpjB>%v`G_Q1tE_E>IGH{dmCJ@NX=@h+~5qJfWz&wPih z2E1lFSHG|0HOrbjGvDp%nzTOQdWG@ZEW1=)9tk7F=_8#FS5hAt6k+m(L#1urz>eX zIOd6l1wZ(2Obhuya3&BA;~0Mq{+QQBjc$YNR=hQOCda^YgH^D1(P+MdU&Cp?-PV7(6QOI<2w=6FS}wD=+J2d( zhiuPja7{%)2L+9CWknXK3LbQYoPICg!@i=+TuqC$UvfX+(EqBYP`(7Pn17^PrV6jg zit5Epl~TLyVcAS|5#1FGT-&PGrXA_M1ZSm=Tm3B^*M8}3kCwN!ZbZ;IDrwGOuyNgU zrgA&ay-#c^Z_i*@V=M!Fp0yrYuu8az{p!$84W!$- zKWq3Muhf3-96nrv?c6zhxIBjsEAkyk^Nak&!9Tx8DDuaS*?Z;ptJOdowrUjIbLRBO z9nA5Yod$5|#Z}W!F={|-n;VNT85`28`7%dWI(`5=BAa=j;tg?d0OjXII`0Lq|0zg~ zR%60q0C*er1p1z^S`S_GYn;9?@r%383+Eu31A54hG*oVWDZReOGO*_;QcaeQKK<~I zUi8iG;1TQuJuOP6^l^2$(0i3rMYMO6(7Ro)+OZATXQcmdLI)uSGKmqIGe6zH(eHa zZodh_8861x*7U*XZxFV4KYzw zvEc0!YeU|IFN0MI8;jXCWL34C{sXbpH3aXCZC(!aY6A+AZa#)3C)%I@n_EJNm?|7* z5nsKd)t?gz9t~z~oXTdOdE0AR4*~{d LCNmR>vEBaz8vitQ literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/plan/window-aggregate-hop-event-time.json similarity index 67% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/plan/window-aggregate-hop-event-time.json index bcdc99122ad..b90403f6a8f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/plan/window-aggregate-hop-event-time.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { - "id" : 1, + "id" : 61, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" }, { - "name" : "c", + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,18 +85,24 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { - "id" : 2, + "id" : 62, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,10 +134,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { - "id" : 3, + "id" : 63, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "CALL", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -166,13 +184,9 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { - "id" : 4, + "id" : 64, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "VARCHAR(2147483647)" @@ -180,6 +194,10 @@ "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 3, @@ -200,14 +218,14 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -217,61 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, c, a, rowtime])" - }, { - "id" : 5, - "type" : "stream-exec-local-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$1", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$2", - "internalName" : "$SUM$1", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "HoppingWindow", - "size" : "PT10S", - "slide" : "PT5S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 3, - "isRowtime" : true - }, - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count1$2` BIGINT, `$slice_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[b], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s])], select=[b, COUNT(c) AS count$0, SUM(a) AS sum$1, COUNT(*) AS count1$2, slice_end('w$) AS $slice_end])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { - "id" : 6, + "id" : 65, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -281,37 +247,66 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count1$2` BIGINT, `$slice_end` BIGINT>", - "description" : "Exchange(distribution=[hash[b]])" + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, - "type" : "stream-exec-global-window-aggregate_1", + "id" : 66, + "type" : "stream-exec-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" }, "grouping" : [ 0 ], "aggCalls" : [ { - "name" : "EXPR$1", + "name" : "EXPR$3", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" }, { - "name" : "EXPR$2", + "name" : "EXPR$4", "internalName" : "$SUM$1", - "argList" : [ 2 ], + "argList" : [ 1 ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], "windowing" : { - "strategy" : "SliceAttached", + "strategy" : "TimeAttribute", "window" : { "type" : "HoppingWindow", "size" : "PT10S", @@ -322,7 +317,7 @@ "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 4, + "timeAttributeIndex" : 3, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -352,7 +347,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -360,35 +354,23 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "localAggInputRowType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s])], select=[b, COUNT(count$0) AS EXPR$1, SUM(sum$1) AS EXPR$2, COUNT(count1$2) AS window_start, start('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 67, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", "inputIndex" : 1, @@ -397,6 +379,10 @@ "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" } ], "condition" : null, "inputProperties" : [ { @@ -406,10 +392,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>", - "description" : "Calc(select=[b, EXPR$1, EXPR$2])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 68, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -420,25 +406,31 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" }, { "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -450,61 +442,54 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1, EXPR$2])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, + "source" : 61, + "target" : 62, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 62, + "target" : 63, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 63, + "target" : 64, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 64, + "target" : 65, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 65, + "target" : 66, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 66, + "target" : 67, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 67, + "target" : 68, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-hop-event-time/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..84a2995c8363ae8f2b8b1a450df82423d2436ca3 GIT binary patch literal 22994 zcmeGkZERat_1a0(re8_7F!s?-K}*ZnYWvx-9f$rHPU5!S+{Ch-ly+j?&GWPKTKhfs zz2_tiQr9xXDpg{9j0vK0C2paee-6D8ia2u}YIBf8Me)eR}JoCw^IZ z@%z2~TX!2Z>zcTbHa8hiMd_$V5j2JJ8l-BasHspw*5f}yZu_c{KCh>@Z!*$-MtZ-I z{?thSOEw!ImxJ0pqF33?;1ja@N#p%*dg}SOk@7}*)g{dP*h*Z z@5_b*M}wnTKeu!&6h4{{92+i6g~AAjPvMv^<2zE3L@g2u214QRNQfJb43BV;fS>cJ zbY3XSqNI5OA({^aJrT;~IDat2@guwsDu#xKBO{)0I5g`Cj?RvHBBSB3hvPy*WHvX< z`6GEkJbwiGdRAf{4vUKnP9}X%@<+Nyc(S?3SOlmX<|w6YRAh$ zMH7pvH@2t=ibRWPK@n+DToRO_#;mkN%j%r00qica<*z6Vt_yFFb*e&cZ-NOisNTJz zP|T?lRHGSq|JM&LY=7d^*{qwm5@c&0%giEGBmIdZbe{T8-1nE<#GN30ye!Sb(2KG(UZg4j-lhXCQmNo$S$h&>Q&BF6JS`$w8VMu-k5A{9 zkMYaL_~m20@ELV|iCUXIi{*KR+}KDS>vl?B02blXwmwTo576?S zyGfr>o|IKl1Hv_O?cEllr)G}`yarTv9~GsXMh0!jMZ7-1dl%Gznp?{TC==-o7NiLf z2qCwhs&j~bu?)kwyRj!sVDw=$7~lo?$S}QY#Kp$3htaMcBPWOpMvQ9-kcE>qVD7_Y zqo__+ibX`7AR7>QN|rT^gl&3^m6{I_wu7#b6AB5gl_YFykVUQ`1bVd4>d>@Ql5=8SWSs)Jey4Az|LMPfv7d3Ty9hZZ z?{=>apa%GcF>>b{E?FEZi15fE-(fY~E2WL1qS8Xyj|#DNT~sL-NRu1D>Tliso-@FFfn)wvsNO_HlH z*?e9``%Rhnff#HP2a?%?@y!0I=}Z=LvWKT8aZ~65nO`I2h{YH`61|+SmU8`bP58z|_Vzth0&cQ4ZC0@&7XRv>M*4<%7+6^q{dgyv>#;m*SUg&9~#_q zq;IXC$%gbYo9o-QMWqyeu<>(_t3 zcD&2mqG;e_;xpeNs{yas&%xJqyk=dKXXd+ITa(uBz1w_C@^5C1Vy?5^C!~EM}4zf@XPQqU^-~0Tb!Iw+t$-nzuftMai{vmACJPH2b;8i{v z;DkUpM}r)fAN6ybe>jlKkAnMoZe$p%Ze9l16g%dv?;_jMaz){Vlt951KCOZE0_r0v z9rlwS8Kqn}AMk_)j`KiYa-K-o@Aq&4ek9`0b36^Be7_EVF!!5~ja|FcPV&kcPP{03 z&3eaqEFEB@1%Q4tWg_bSITwkC=`txJ&%kM0mUii%Xd)GZq#j_7V?rLK5 ztG*xZpqYmU{_(LlEBfKyh2o0lj{sjJ*5Y)&v`qs+eubZBP+Y+pS{k4epZ2X9@()axK*WibF z6Gor)n9-hR`>XAN>p*ol&{)USY!lSVVD2rK3UW3DW#5EUsbLYjdG| zh!cWgKkp~x<~iUAQ?cEooes|C^)RkskF_%~t#|HH#d@n)u}&A5pDj{B*G@mDoPpe29>_Y+00Tox3n$%^X5 zk9Vb}PSt5l`jtbte_D0Vcck|NT#D=X>5^sO+AqEBQSt`sW`r;=NSZSlY_5npOS#<* z(GP7YZ_i}dWNZVpP%y6N!zXM1U9m&43FG>D=1?+5*0H<;bJK)M}2ZR39X z`#ffFZVBN1u(dMJIpn%Dhg@rtz7$;+mjL2GtmBJxX~{nN!$!3l?ZQ@#0?pK?`_r=k z1MJkvhtg2fPcb@0Yach313%+?^h$Ob>Pkfq!p+-eH=R+Zhl|E^UKAF*5M5#hQlro3PprLqOCx{dDr#JD=h&K-3V7#4y&-xcTLW`VmXRH%C#>Wa;>04?O;y zcOj+&47LWa9#(B2p-UF+wwttP!@Y9O7Vl|br#`qgx2EaB3|{?ScHai(vQ4(l=V1M9 z?N}4MO@wNwUdHOc4@^cOyh1WM-Vj!&J%hW&j584B`7E9q;bW&ybsdN%p{SWQ)%dO5 zd}_edtj(~x-W%b4EJA(xgcE8WLE*9w6|`#w`SjMVv9Yn&|FN~QwJpZ^=vwuo%OcP1 zH$hx3;kT~f`En^jhJ3L2Gnbuy*O6FOrjJ?Od}+_V*Xe!s`nr8KD}k1&8@P={*KW&Q z)r8?pb;HW={-fwQQ@!3Lv{Pa0{*R|R`nv2qbCUUG4E+dR`i(hGsxFIq*)uF3{S|kd zS*2PK%1!N@dFWtMM1+quy;aZB|KkDKZL5Lq9f!;NaH4R_Fw{1kCW-6fMFw~0D|H<) zQB|>^^U08y_-a*s z<_8tF!P$(Di$JJ4e=AH#n_*|M`MC#P+;INW1JAH-a?A<9BnYGi*SD!X1==^lcs_zjSN7Z4e2xPb{s zx3@~S*M@FWm|6+P**hMcoSKY}M-yQEGe&r@3KIwbb}WmIYa@!)1~>|~y|L1S!Q^@& NFjaawos3OR{12g(FP{Ja literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithCDCSource.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/plan/window-aggregate-session-event-time-two-phase-distinct-split.json similarity index 62% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithCDCSource.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/plan/window-aggregate-session-event-time-two-phase-distinct-split.json index 33feb6a5e17..d6fe8a28d3f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithCDCSource.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/plan/window-aggregate-session-event-time-two-phase-distinct-split.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.20", "nodes" : [ { "id" : 1, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyCDCTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" }, { - "name" : "c", + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,16 +85,21 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "changelog-mode" : "I,UA,UB,D", - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { "id" : 2, @@ -94,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -104,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -117,8 +134,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { "id" : 3, "type" : "stream-exec-watermark-assigner_1", @@ -148,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -168,32 +185,10 @@ "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "condition" : null, + "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { - "type" : "UNKNOWN" + "type" : "SINGLETON" }, "damBehavior" : "PIPELINED", "priority" : 0 @@ -201,14 +196,14 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "a_int", + "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", - "fieldType" : "INT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -218,40 +213,19 @@ } } ] }, - "description" : "Calc(select=[b, c, a, rowtime])" + "description" : "Exchange(distribution=[single])" }, { "id" : 5, - "type" : "stream-exec-local-window-aggregate_1", + "type" : "stream-exec-window-table-function_1", "configuration" : { "table.local-time-zone" : "default" }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$1", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$2", - "internalName" : "$SUM$1", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], "windowing" : { "strategy" : "TimeAttribute", "window" : { - "type" : "HoppingWindow", - "size" : "PT10S", - "slide" : "PT5S" + "type" : "SessionWindow", + "gap" : "PT5S", + "partition_key_indices" : [ ] }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -261,7 +235,6 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "needRetraction" : true, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -269,137 +242,134 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count$2` BIGINT, `count1$3` BIGINT, `$slice_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[b], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s])], select=[b, COUNT_RETRACT(c) AS count$0, SUM_RETRACT(a) AS (sum$1, count$2), COUNT_RETRACT(*) AS count1$3, slice_end('w$) AS $slice_end])" + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "window_start", + "fieldType" : "TIMESTAMP(3) NOT NULL" + }, { + "name" : "window_end", + "fieldType" : "TIMESTAMP(3) NOT NULL" + }, { + "name" : "window_time", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WindowTableFunction(window=[SESSION(time_col=[rowtime], gap=[5 s])])" }, { "id" : 6, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `a_int` INT, `comment` VARCHAR(2147483647)>", + "description" : "Calc(select=[name, window_start, window_end, a_int, comment])" + }, { + "id" : 7, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", - "keys" : [ 0 ] + "keys" : [ 0, 1, 2 ] }, "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count$2` BIGINT, `count1$3` BIGINT, `$slice_end` BIGINT>", - "description" : "Exchange(distribution=[hash[b]])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `a_int` INT, `comment` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[name, window_start, window_end]])" }, { - "id" : 7, - "type" : "stream-exec-global-window-aggregate_1", + "id" : 8, + "type" : "stream-exec-group-aggregate_1", "configuration" : { - "table.local-time-zone" : "default" + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" }, - "grouping" : [ 0 ], + "grouping" : [ 0, 1, 2 ], "aggCalls" : [ { - "name" : "EXPR$1", + "name" : "EXPR$3", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" }, { - "name" : "EXPR$2", + "name" : "EXPR$4", "internalName" : "$SUM$1", - "argList" : [ 2 ], + "argList" : [ 3 ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "INT" - } ], - "windowing" : { - "strategy" : "SliceAttached", - "window" : { - "type" : "HoppingWindow", - "size" : "PT10S", - "slide" : "PT5S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "sliceEnd" : 5, - "isRowtime" : true - }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } - } }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } - } + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], + "aggCallNeedRetractions" : [ true, true, true ], + "generateUpdateBefore" : true, "needRetraction" : true, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "localAggInputRowType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[HOP(slice_end=[$slice_end], size=[10 s], slide=[5 s])], select=[b, COUNT_RETRACT(count$0) AS EXPR$1, SUM_RETRACT((sum$1, count$2)) AS EXPR$2, COUNT_RETRACT(count1$3) AS window_start, start('w$) AS window_end])" - }, { - "id" : 8, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "INT" + "state" : [ { + "index" : 0, + "ttl" : "0 ms", + "name" : "groupAggregateState" } ], - "condition" : null, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -407,8 +377,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>", - "description" : "Calc(select=[b, EXPR$1, EXPR$2])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "GroupAggregate(groupBy=[name, window_start, window_end], select=[name, window_start, window_end, COUNT_RETRACT(*) AS EXPR$3, SUM_RETRACT(a_int) AS EXPR$4, COUNT_RETRACT(DISTINCT comment) AS EXPR$5])" }, { "id" : 9, "type" : "stream-exec-sink_1", @@ -421,29 +391,36 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" }, { "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, - "inputChangelogMode" : [ "INSERT" ], + "inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ], + "inputUpsertKey" : [ 0, 1, 2 ], "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -451,8 +428,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1, EXPR$2])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { "source" : 1, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..cacce2e5533e01e3191dd2b0b4cf2dfaf3279121 GIT binary patch literal 31910 zcmeHQe{37o9e=iy^mo%j2ei;CTGNg}{M)hPbrW+^w=uX$6uT*u33EQ5lWT3Cdv{LK z{=rg#0I6c^2bh!H|$Pw(k3VchA1d z`BJB?O=<2)_PcvOzVE&7`~7{N@4M8U(_TVI7yQt;G>fcyfB(4r(VK@KzAi3pd+)ms zF+b|h0@1Lm^ccEh@AEG5^`cl3KIfn1WPedA6$DxFmxKOXG;}l)&jkY~jtxeS7DLCv z)ynjAEP!VEm_O@3Qmcq+FdU1-1Cdy8Fdm2v4F)2?!I)p+W`(LGR#dML9}EtKywTv` zAQuRR;(qKK90oPxGg@y>~rf;8t^z%w>|3sYE(;I5`QvZt_J-0TfjhxH8a%a)~~vD#)BF$%-$Fi8r!u z%It=}V70pq8SW%&@|H4~E1^*ajYG63hbXi98wKje0&1=eM^h;WOcfx^3KiAieTR6z ztw|&2CFo^>3@r~0wD4P!wtsb$s;qIY-WKgMS4m?Wm=tCsjQqzOpc5uCck|4>{*AM zT0dLVs*R0LCg*6GR(6J3>J7L|fWf+4HMdW9ima*10x#lHEKRzO!umobk=A-=o#rhN zYtIU8;MZnTFl%NL2PSfdQrXe*gV`Lqo>ibroO`#n&yQm4$<1~Afw8#;$E~tXy({a`ggXR|9S&n zR0JD$Ln&}~<01AdrcN#H7q6bmT(t^SqBoyQYW379mUJo|`^scP)U&xhSuNFke6dL->bAi>#%2J-p|sD>?bgK8^ASY;<53yECl0 zYtow?3-D%0%gT+-5Wrlvvu7Cv^7QEq*S`@i&Wz7}>yPuNMqk3P@LRwDEI;^cFcS|H z-~LI@KL5Voop@;DTi0g63q#V-Hc{Sr_l~DO{Q35)rydV4j6L_rZ}i{`V4G86%%wi) ztC@k{vZwag-h0|Z*P-fgU?~px-doCdCHr+;nj8!k3VbojdxyAC-W!QWgWi0Y8}x>U zgix>;3&-Q3I3fNwp;9^^P4+l@MrY3mqAA0cSsx8tZqg3<66u6|se^mQ>O~5;J>y!;=a2KBYT?k@=|3Mj?O_YIa1;jMJ1clD75#Z(B4m-_N2)={iFbsR7t2c zxusz;ffI#Une4X7UI7Z@Ysj+<>(KcsoT@nI0*zCjWgoE^1ldN&fUE7}l+$0}tnb45 zi{Xp{&y)9@`tw(R@!-z4+{AS=>7M0EH31F-HZ_je#%krfAm4lDo7>kt^!6zn60rOD zZF-$Hoai0fH!Mzn63!(%vViCrPVI-INppW`&#CmsBi+2lG~4DXaCEEzhnARl&xw|B zI=s{2FFp{#>F|Q@!qGpg0qT=MWZ;E#@OC0-y7jg1Wi?rK1Vp1xRyzM21k!~dket@H z-2H{Q^*Q{zF#fgHdndU$+*h!noNnX0`4V^HMDxx2TBQyvZ;A*6D2rbMwNh9h#hmyt zg@yoQrJ{~JlZithxUiGRAP5MB7qg-;=L3-I6xfTU7e1TqS3MpAHyfOOX5zV%U$ioIm6ZQ^#-}~a;M`}263srqIM4R=*f5!9f6K)&7Cvc7SBHnt-di|J4g3* zjIk};TRXZpENh%IT;p1z&Oq}T!!X+;2-8EX;@^v>ytczMuWP!M&Xlex5I6M79L=bv zN@mWK4w~mJs{J`rI#vH(X&PLYXSF+1I#Wh8Pu=t`1KPyR?j4=dag91~68`$|2~A$J zPA>fZ6{^^>`X8E&&hAkW{!uxG8W*VS6SBMI=%0ts7PXfyjaFSD1eY;r&}IlUeAl&C z1~$NRA3C7nH0$JFAJsDX(kr%^#jJ)~j=Y9x-2@?8y}*>l1K`pJKWdiBK%GVig4Suc zXvucBOob`==*(_8o{8BMyAHVkb#Q9fd*kMj=FI*QgznR#G+IUr%yvN>{~ub7+&wCk zN)m)NFp0LTmMNGDi&4=qYV4ian7Jk`3!^N&{{8zu`e6H2t6zVo^nBOj_kt6KUVtBc zKzosSa<6;Ov+r(yxbIgx&OWgFo)^H|62*m$9By8Ln_!N5WkHoWUIkmJl~la{h3ch& zb;CFC1?*Qf`^I%-y&k0=v{(1quiJO)_Rns>CllH6N&V`Py{SxgGBtcKn;IX>9T?x6B%3l5=~OmX_dtDSyBm3| zyDU4=uoZIJZ*?*Z`O8D0?h}PlLBaV0j|x6`^2~KVI{7FZTDa0=V-fwB7XV5(X@F8Z zSMsL;=8K0(mpD>&n^h7H#iaK%Y0Z*&+@uR|v9zqi$MIGcmGl}43a3GPZYRCE|AeH7 zDnM*ia^>v?chvr>DwRqR1cj-V!maHDQ3jh$5yD7vH+plj#DwUwhpemLAgO##W z5Q`$cbp_h>N4>j(&;0!>n`kfSDuD~*-N3mCGzf1UCJQg$@!<|i#>K1TDnku7f@#%z zWrK1;mbv5DzvfT=di%FukS_if;`ah3xdMSVpVI@R~Xu~hU3OIU4c%49Jm&qku*Mc%~#>#7wUD< zkljdkgX=D|Ho^T3hW$QmyxSnb_1fKD2y&EFNcfTw>YMZ48DaNkyyQZYde#}VR%eh< zDNwh57v>D-Y7ai#P!H(gE6BhLL3)_WtRqP339M3T<(xIIk0$Vnxz^)mpC$XNv zUvpR(3>>7z4bS8AR_bz%smIs>0JI)suabfE+yEdjfV8##@z_6451rWa)G#a!VVK5X zJ7^aA@B$zDeiKByn)YrLwsD}D2k26mR0WjBPVGMOy(e;KeH8l9RRC9&d`jzs7#PlK z^uX8+(WQwu(!ntV({5VZH=)ny68@0Ei-AzbWOV-53%;2aoZcG+)hkR3@)du2?DU5x z?;C|fWR;H(T%MS9Tfy!foZv@DW)&ixRJW2 z?dqkSQ%>((O6@_Desa*Xi)GG^02Ku?jsUeVeB2SBUhEs59RVtpk2?a?87lR11gIwA zH%iASM4}mN)DfUM0#vx$!x5mi;k!1D0M*fAfo*8#ZX)c6e8v%=UTRt_mvyx}0@U*q zVL7*GAmn4Y%yR^&i)O6lGSCsAY9BhO@465@XwYZ3aW|kBQB>$z@z)U`M5~T-&*K{* z7Kl1SBQHiNL)dy;jz+b{W$k^ECdY; z50CXa%2OX}+hF`)>9co?%bw3CF3W>mT;|`bJ+pK;U7UPjk0E~^_HFF^wY21{>G;t2Uld|0)W{I;bJJrb9p{m6avLU zF&5>A@$kuVRreg}TQahrT&*)BH3U$#KUnqJXEzdFhx z=GQlTTooo{u?!INS%JcU4RhyGo`wn3(0ZXvVDzQ%kR>-vtsA~-%pawnOZJV_A17q| zo~|kiy7~ukyUFUqWD^Jsc}^~ja1|a@ZiK>NdgC`b?c?Ap&hXx=2iYf9L}kY4lBwkZ z6N)a3Os1w(0jY{kJw*vx6FjV`Rp1URkjWJQB;Uvzu4kLxSgjYs-w5hw?7%nxjihXF z5kR{JTVXot+#+ufZC{p1O=$8WYn5evr-%L-46$C7D~Vi_*_HS!UW8>ubyvTlYCPa z!XK;`K!w177H#r1&^058vGK9gNFoi_nCUg#SEmUX0Cp_;oOY9@SdW0U(6lthJRwwc P4Zxu@2Qw4NvAzEXwu;k* literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/plan/window-aggregate-session-event-time-two-phase.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/plan/window-aggregate-session-event-time-two-phase.json new file mode 100644 index 00000000000..758216edd5a --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/plan/window-aggregate-session-event-time-two-phase.json @@ -0,0 +1,491 @@ +{ + "flinkVersion" : "1.20", + "nodes" : [ { + "id" : 10, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 11, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 12, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 13, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "SINGLETON" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[single])" + }, { + "id" : 14, + "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "SessionWindow", + "gap" : "PT5S", + "partition_key_indices" : [ ] + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "window_start", + "fieldType" : "TIMESTAMP(3) NOT NULL" + }, { + "name" : "window_end", + "fieldType" : "TIMESTAMP(3) NOT NULL" + }, { + "name" : "window_time", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WindowTableFunction(window=[SESSION(time_col=[rowtime], gap=[5 s])])" + }, { + "id" : 15, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `a_int` INT, `comment` VARCHAR(2147483647)>", + "description" : "Calc(select=[name, window_start, window_end, a_int, comment])" + }, { + "id" : 16, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1, 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `a_int` INT, `comment` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[name, window_start, window_end]])" + }, { + "id" : 17, + "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, + "grouping" : [ 0, 1, 2 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 3 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "aggCallNeedRetractions" : [ false, false, false ], + "generateUpdateBefore" : true, + "needRetraction" : false, + "state" : [ { + "index" : 0, + "ttl" : "0 ms", + "name" : "groupAggregateState" + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "GroupAggregate(groupBy=[name, window_start, window_end], select=[name, window_start, window_end, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5])" + }, { + "id" : 18, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ], + "inputUpsertKey" : [ 0, 1, 2 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 12, + "target" : 13, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time-two-phase/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..948bfcb9e520c75110eee4ea509e31e812e191f5 GIT binary patch literal 31476 zcmeHQeT*B$6(8Gom+xE>QXrwESLG6Da<2IuZ-I~NEQX1!kDZDMif54?kgo;8+zoZHk_=8eG(-w&$RqY@AQx#MysHiHHwkotjKq<79sG61*rSHw| z*t=f)&X=5>knBkI?#{=XnKy5~elznXcITv<5YhoZG!E@Smi_3DJKoM6c;Hm}wudHm zzW3axbvNqHJW;>P^bopb_s<;Up`4f(Zt_lZvNtE?vx2O63qEgp(0?=-PW$?2j|~hS z&H0Z7O2x^^p?)-z$Gj=;k#bQ~eMA0%p-^aGaIn8WI2a6sgF&ytO$#MSEUIq*kk9YW zxC3E6+@Iw`f&M_q%MT1?!kJLWE%*aIchHyhxx+lqxr6>-xIgR*^1gupA?r6nI=e8f zPLh)3Ne@@z_$k4Y%ZtUM9+k`F1&>@Vs$xO#s3%H7Srzk&Cm|i*!>L?SkVP&p&I)p` z#U@$gO3IX^0%aGHCmRoa2>%*8I*BihB;~ZCa;o4h$r3LpiddXXEBKU(?&D%HD;;;M z(>XWwsX$Nj_)EfTGF*R{*+@FVQ2F1esGMS@EPW z@@n!;iCyy+mb%lB;C8YmW66WL5UP35I7G8@h%&3UnxS6Ipk`aOH083-Q~<)XP*iQ+ zw~6j*h7K`j$xkUTUX;kQI@i|$D$Q;}3Oow{v2N;;{`gQK&7_>WtXk3!E&g@RNx zRNf4y2%fl9oUBSlulds?C>uv32cpUG$cUx7O<3Y%Ed{OBo(W0Jj&cP-fhI*TMApFG z6zkP>Sj6!>WlnwexDG_d=V6YuZD`D#16-+^TvM{Nw!vX)8(PEmRyM{RXWk>7icI<} z1*o+(*w8DUyfi8DT;3BaswuEwtalPJBa{B7g2pux=HSl5WTmK#mh*Y+UE*XpMwgHz zRV9OQAnVn-ATRBLyfj5#nj){BBCmlghTAn$a@-}8;9`@6Vf9>*4K!D4k8snRC(l8D z5iOPrM#qv$mc>R>(S6Z`&b{VXFu}l!nFaa9tLdh(j&KOxyk_BGGp;(-B9EI_PN`lSR&I-+2 zAk?laRKTxG#h}+rMGlOo55-dZ#}1~_Xp=rXHX4PJbwPWz$Pz+s8fD@bxr=l`qm&ps zgp>_kQU@8e6n^8*4x0CBXV+``GIf22Ei}yoQ*5HO*+Ej3VZ>cCL8xchR{D1~o&Jgd z*JPoAy&)GkyK#}mBc^r@?rTR+1+G+rBGJXC-r(}0jh@=Yl6J+@xbRy!Tt>m$HaABc zW;U5|;uomZy@SUNyq0Rf=kG%BnAcvwyWdL1SiE2J8 z|NnY3@Y-7&9)9DIS=aU7JF@Qs-6!?kAjpTWKiK6ge17*0cYoU!-+$Gf?i=){J~MYes(=4pdwQI}XfY7enNw@AkE-BE! zNJW)gb-N+aW0@lYuR^Nz=~7Rf6({NrU6ko05@sz&9?KDFF|+Q;Y}Y*E1sJ^H5|^luGZxVnND^Igw&g zAlLuv-sXGu-3QmvTF{XP8^*i7g$Yy%ZyF|ZFWzx>3nk;=RdThVgsa}P;=QU$nRZ@S zzh+Lpa{afS=f3SC02*~Du-GzF!nW!-jv=}rKLb|jrmHqMAFG0Rx=i3Y&0eh0y5KvU z#07$%6%<|;OPWw`FvzqvYeShXLZv_sTnFz+8s9r+N&t&dCrHTB;9*F2jpHtStb^ID z0ob6zyBQ)}t*w0E+L5e6#EYCro9o6>LLKI`w+8}Pj}Sm~-@$k!QDM+Zok2n|OYM3c zm@@3DUHG!2;?czyk%8W(L}*x?tOEs`^V~HY+XpCVC4yy2xsVoVBz;XqvS17hgsnue zIt#T#Eb?j^GlR+T(d-6h@O=}txz{@jLDO|J>73^BWdV8)L*_@zg^VEId*&P0uloAy_g6Eo^S_eu^;LdB@d}fIyy4G} zJ%0A&sSuEkG18d8Q5t55GB2!!N9!^A%8J#vli!(^|iz*P=_Nn{IZfVi&u<0mwX*#J+~E88uvyFctfS3@)zBlF|oJdbK(4anqH_ zJT^Ymxo1n?Vs|;{(q~vt>>{mR`)l9YT>D~b&9 zsWfxAm%^}dv4Oo|_aZ`tF47uU@}mQwVwYLj{LVJJzy0V-bQ}E(FKJKMbwLeo#7D!v z;k6Q$qb}rRZCHBjAKc*VBQ=Jd&y}z&-nrjt@h$(G_QWDWgS*hsuy1&+gl!Zr^IQAG zu*~j!!m_+K3d`JCKG}CO@iM>l!ZNo;@v<#kyq1m%7<)-pwZs!x0rJ+$+=3Y;pS-j_yUq-i4SXMS`YVwV|S}5N6!Y&kV zi|U}B3w(Tc_r2wfvp0Qn=oR00ez11$pTG(;GzULpsp2OyYkvE}wy~*KpYUGAzZ-w& ztJi{k{UGeH&m0S>J$FXT-%I_q^UbBEXw$PfurLSQw=C@E%Jk9hQ$5+-KrjfZklY!6 zkaq_MLhv>)$h$)W;VjRGGeWK(wzzxxp)_a<+gc4iE*+O!njP?0=)N7pPJbKbTma^5 z%W|&`b6#pN=jB=KhMi_Ed#BlX041Gd4O_yE@O&COJv5h;nukl}sB7cXJ(v73*6j@r zP{D~I&Qn#dq7^>vE4<)= zvudhvLCR>CgXY<)ex~{(>jf)#I~2OwTSXQ-!n3!ET)<8Td#lLOY!%V*&gM>fCb$~y zKA=shlh1Rrv-Nc-tyC&n`s*rK2AyhNO-DG@vP}1GcyZ)iMIx?LC9bYS%nZbu>gLo^#Xcnqo^QVdY#hF2EHC;Yoxc#FHjn(CT--o?gXdjBG z88=j$*j8wjY0D(gUM;j&3tOw_UehXL7=YNTh3B)a+6pu--K{-aO#Qlcsg&KNYd5D= z&;Qt0!qmN8$FSCBZve7409o%LRFNr_YpE*7NN~1l7(iXlSJzygBRRWEXBzct;DL&W zo9zt~c-)rnBCYJwv1aw1gufnqLzB@SCu@%cp|x|r2>hrI(QI#v+Yat z$N_v?ju>H{@e&4R^y@nM@aRSlv*ffF%b6#%wj`2}K+}O&uxmzqK*^9%pz#3M^uUjr z`4Ui};Rmk<8V>l}-ZT|cNljBQWfFs;Vbs`ngG{1VeFPG+AZ={04C*rYqxAM&&q(EU zT*m8JRZ-Bd%OFlCS$>$TgAD^bCuc{vA`cl8vWMx8J+#`(!JWYH-K~4sD;7m%%4m}; zXF!QI+AuQ7vQh%o-e}d8li&s!cv(>{!WsayBU!jrpqe)98|&_%(X-*LdiAnq1;cY1&-b!sgeQfr3kzv3tnMH;9==O zDtaq~_Ev&1{qriRtDAl!+@hdX3n4-^`m1EWHL7~KHm185G?Q5<0WaWsf`Ekym~Kxc z-JUw>rYwZNW-gapF%lUa8;y-b;;@XPmvB#oCL{pZQFA+U9Z{?Vz*4B28)KSKxAqFq NAD=v!9FLCf`9GrRORN9@ literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/plan/window-aggregate-session-event-time.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/plan/window-aggregate-session-event-time.json new file mode 100644 index 00000000000..ce859d63d9b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/plan/window-aggregate-session-event-time.json @@ -0,0 +1,491 @@ +{ + "flinkVersion" : "1.20", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 3, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 4, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "SINGLETON" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[single])" + }, { + "id" : 5, + "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "SessionWindow", + "gap" : "PT5S", + "partition_key_indices" : [ ] + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "window_start", + "fieldType" : "TIMESTAMP(3) NOT NULL" + }, { + "name" : "window_end", + "fieldType" : "TIMESTAMP(3) NOT NULL" + }, { + "name" : "window_time", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WindowTableFunction(window=[SESSION(time_col=[rowtime], gap=[5 s])])" + }, { + "id" : 6, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `a_int` INT, `comment` VARCHAR(2147483647)>", + "description" : "Calc(select=[name, window_start, window_end, a_int, comment])" + }, { + "id" : 7, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1, 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `a_int` INT, `comment` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[name, window_start, window_end]])" + }, { + "id" : 8, + "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, + "grouping" : [ 0, 1, 2 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 3 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "aggCallNeedRetractions" : [ false, false, false ], + "generateUpdateBefore" : true, + "needRetraction" : false, + "state" : [ { + "index" : 0, + "ttl" : "0 ms", + "name" : "groupAggregateState" + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "GroupAggregate(groupBy=[name, window_start, window_end], select=[name, window_start, window_end, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5])" + }, { + "id" : 9, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ], + "inputUpsertKey" : [ 0, 1, 2 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 4, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 6, + "target" : 7, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 9, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-event-time/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..7ab277a95f0f9b44b7c79958ad145cedd11f2746 GIT binary patch literal 31472 zcmeHQe{37o9e=iy^fzgt16pVmt!c+-{o9V?RTFbkw=uX$6uT*u33I+XC)e6O_wJme z{ez`q0wly>8k^W31BR+iY@LP#NC;`0G$BA@f^CRN8`EIZv`uVG8$&|c*t+lc-JN}x zyGxulc1v?lvfthN{l4$}etqBP`z~|$l%EjN1wS+%)go&?I5Z}G{JUe1-Ix@&|MT0A z>VDK;4@ASR(PC(hy)SvlBPF5Ce=az~NWqd==6FdCR>Hx;K;%R$SqO(tpNtQjC`C?2 zYt^Z#L!4DIR^=!qR)4cX{a8WOwdrW8acSQn+HbdE za#n<1#!J9?Xn@0SOW%<`P{`#|`E((F$9S4_Q=SJ;uLJv!TZ)fN&x)0bST%IsA|vyG ztXQ3Daz?-T)x;=UM^lH>x$)GnrMpd9^3z8O+N%STqQH$Z6<&rRg*Rm8;BLzGS{o8E zEK7-FPafal)c7LAv5pPS#5v5=n#DC0OC1|Lrm>+tWN+tS+;#4K(k)A5pQQoSu?7cv zIZzg-1ePfWGF2rH4m#<*gv?1K)Kb)}nuIyJ_ZV3($fNag8E2O)S&P|Ch@zs9fh;5& zRHvXIoq~e2L_u1jpjM)wAz6&Kf3D`aM6CiYmEexj5IvWNM;gvGNh->SErQe)k@0A)Wq;eYT`@h&!2b6srIu4 zTy11Ni7dgHR&qvK>J7L|fWf*_GxMi6L)O(Ko)vH@mL**$V11#GSZlqrk>+g>YyT=` z@Eh_Om^JgM!{ddcnf%Duk$eGN3dhDq)6lX$7_XMuLP$*`Pn;z8kRBM6CdQ6p%7!Va zi;P+t-+8iw;k`N8^_$*IQ{OR5n&yQm4$Pa93s+ASrdER{(b&gsae2vBPn{x3r}A-b{B|yvk@0rV&l69*_l?+5 zcv-8pHCX7jO|xUSQ+HUyk*nt6&k(HnvVm^2cVkF{zcHD851=QQb?`$m3AsrXA`qgB zJ70nBwfe_l_4Ja=}}O>aj_(_^#W`2F02Bd=mu_|0Pgy*&7AFz>y8 z*K;5JbjNj*PetcPUwr&m;AJ00r_{V5GCb~0ncr9XZSS8}Iz=}Y<9!0-$&ZLL+{)Ux^p>olTDl; zD?*i3QY>4*Ne|6swMDjW9GxUD={Yz;sxoDRK*#E2kXQJKcA%k6!h0QLBq-aF11UwU z2rLRkaNOAbK<=~uKD{fVo(Jq+G^QTs<%j>Z^>OdL6&TsGq?Z+|GjIkDBF5n|Bg+ce zl12G@5asVd>a;&gHfTbB%u*qd)*fwvSV8e}QAWi|a3j;|4q$-v8hqANkqCyWa5<&#k0;hAG#1kn3-5 zo|cW)D@9(q@7&jSY`2u>Nz{iQtz z%$G(`dA(`2EmYwYSj7NoL$BS$yCm?z?eK1gUwKS{+u`}Z#RGZP0%|9RAis<0;GF=& zTv3x%NANJ(WTovLTrknafQel0TkihiLbz{&nZ+AxcQzD_}Kv9EhoX&nguPoeF!*g z9~$uxL)fJB^W!hx`Yd~{pb8|0ERmfdFy26 zwkT;gaD8)Tbq|Rfz*I+!uzE;rSb(^vg~qi+gMj8WhGDix2%nEw1&u4G@wC%4ud51q z?v$=-oi>$`Ty+{-%_Mh92X*BZb;aB%o$3s)HnpECv)SD#UDseUPu)}jp0c>vy`xh) zu2J`$fxkX{LQ_<&GxNWBooYj@{kv+Tvv)**f7Ga;#shlZgzRZKdJIDA@TG=s0^VeR z2A3~r&_>+}-Ds)~`!<0?pYF)04`=@Jgj%>)UtgqflHs7 zFCmK+ppgb(l#OY4;B`;SspuLZ8j)L$XCg91M?T)XzI_sTDIJ#`=WV(@tkMdRUv**tT=yZ?pvcRbeji=F2m z+Vka?!PgSSgZvLSdBDvgC;SqxNDQlhE!WEm-X}u!oWQ#IYxn~88>+o`1KFrWX$0-n zybfvhy_)?K&Hj&S?@MSlO&_}d54HEdsIOC8LR0F&rs@cED>;Jm7z4+G1OHNZ;6adLoq{o6s(@ zIFQNZCo)4v@|m&G!r`$4X|g#tp3US74G+|g1rUhs{Vvw+my^ucihAs~IvM7Iljbt?SS!yyHV5y@rayfzJLrNw4NVF3N%eC{Kl4bEg5}IaWN*vkE{w zHTCj78*C@$8-%-ss46*7x27efXS&UhX%;#;&y6thG-gjvkXk#FhV-!=wYq`Y0kmTe zExU|L)}G5xg}ozdSs!SY1Bh=AMfoBB28P9Q8JY~5Qtm%et%#gZ66hTsz}Fx5?hZfy zx36rb)axpPi%z?JODE7wc-s(}f9tvJ1f2%49l;-#zq@Z{N#&fGr;qhdzKUt`Qs8Ye-CmsI1U z^B8^9o7H^&-A*W7kE_e+>T-5cm$M@<$Ii@4^eP`$mlKy_m&a{ym#Wz>W^vJ~##+bD z(L2&XF9Zo{vu;|Q#M9S@+SKiG-*Vwp8uu9vVtU%H;n+F-=+5QT9+>GT2N1hZ;_M1g zQ6S?AP`RN~t^jqV?+bPXs8l}g3Q(u1)XNp1nuOmd9i!L@J3Of?Ky?MEaJ_^pKyAZU zQd|M5tKI^K7oEEduOsrAOC?FWY|c4Xr)23;HM=Xqx+~4*^I`n3(NPVol;ZfanM7gPb zb-xRVtCztkuI_Kg&w9QWQeQn?r?`6Cpza46o$m0sWE(9Ocd2~a)CXxD)W0`Bxbgch z|F8F*nOA@O)jQw*^i$x9gq#O3I~OfQ!Yor{2TFXX#FY{Q>|l{khT}!97>O{bk+M+> z4i+_f8&FLI5809v>k`XP@CU?io3hXB?!+zJtL`j?0?`jhv+x%TYsTVL=J zfJ(22GFBM+0ZE0#tq6Zsl=vV&$Fr~o605;RLI)esbTG9-Y=PEWK_x(GHbRQR(-kse z&C0IU#&Y+=F`FeNpapy{AZj53mOIcWcfh9HDl1X#Fi;>cU`1`dEV*ttH99t$8BS&4 r9x|i`U$JCYkuj~@6xv4&A_ literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithCDCSource.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/plan/window-aggregate-session-partition-event-time-two-phase-distinct-split.json similarity index 70% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithCDCSource.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/plan/window-aggregate-session-partition-event-time-two-phase-distinct-split.json index e476afdb87f..c7955815d9b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithCDCSource.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/plan/window-aggregate-session-partition-event-time-two-phase-distinct-split.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.20", "nodes" : [ { "id" : 1, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyCDCTable`", + "identifier" : "`default_catalog`.`default_database`.`cdc_window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" }, { - "name" : "c", + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,16 +85,21 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "changelog-mode" : "I,UA,UB,D", - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, cdc_window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { "id" : 2, @@ -94,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -104,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -117,8 +134,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { "id" : 3, "type" : "stream-exec-watermark-assigner_1", @@ -148,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -170,10 +187,6 @@ "id" : 4, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "VARCHAR(2147483647)" @@ -181,6 +194,10 @@ "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 3, @@ -201,14 +218,14 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -218,61 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, c, a, rowtime])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { "id" : 5, - "type" : "stream-exec-local-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$3", - "internalName" : "$SUM$1", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 3, - "isRowtime" : true - }, - "needRetraction" : true, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count$2` BIGINT, `count1$3` BIGINT, `$slice_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[b], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s])], select=[b, COUNT_RETRACT(c) AS count$0, SUM_RETRACT(a) AS (sum$1, count$2), COUNT_RETRACT(*) AS count1$3, slice_end('w$) AS $slice_end])" - }, { - "id" : 6, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -282,48 +247,77 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `count$2` BIGINT, `count1$3` BIGINT, `$slice_end` BIGINT>", - "description" : "Exchange(distribution=[hash[b]])" + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, - "type" : "stream-exec-global-window-aggregate_1", + "id" : 6, + "type" : "stream-exec-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" }, "grouping" : [ 0 ], "aggCalls" : [ { - "name" : "EXPR$2", + "name" : "EXPR$3", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" }, { - "name" : "EXPR$3", + "name" : "EXPR$4", "internalName" : "$SUM$1", - "argList" : [ 2 ], + "argList" : [ 1 ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], "windowing" : { - "strategy" : "SliceAttached", + "strategy" : "TimeAttribute", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S" + "type" : "SessionWindow", + "gap" : "PT5S", + "partition_key_indices" : [ 0 ] }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 5, + "timeAttributeIndex" : 3, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -361,39 +355,23 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "localAggInputRowType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s])], select=[b, COUNT_RETRACT(count$0) AS EXPR$2, SUM_RETRACT((sum$1, count$2)) AS EXPR$3, COUNT_RETRACT(count1$3) AS window_start, start('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[SESSION(time_col=[rowtime], gap=[5 s], partition keys=[name])], select=[name, COUNT_RETRACT(*) AS EXPR$3, SUM_RETRACT(a_int) AS EXPR$4, COUNT_RETRACT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 7, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 4, "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", "inputIndex" : 1, @@ -402,6 +380,10 @@ "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" } ], "condition" : null, "inputProperties" : [ { @@ -411,10 +393,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT>", - "description" : "Calc(select=[b, window_end, EXPR$2, EXPR$3])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 8, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -425,12 +407,15 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" }, { "name" : "window_end", "dataType" : "TIMESTAMP(3)" @@ -438,15 +423,15 @@ "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -458,8 +443,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2, EXPR$3])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { "source" : 1, @@ -510,12 +495,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 8, - "target" : 9, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..12751703bbff0af299c37fc75692e54938eb5bb6 GIT binary patch literal 23641 zcmeGkZHya7b?o~{E?;+PL)-KN9psWi6WeF|>^l=s@m-w6t#fy>eU~WDkMIB&<~ZWMU)DO{Ag)GR238il`5bmyf?e+ z-96hEUm81!ZdbBrcfQ`dH*aR%ym|BD_Z2*ZkWTpFKQl4+%|Fhc z-2Z}}wutFQCVaO%>%V<&7!!{G&cd!kmkkr9qL!*PeR#gZFN^ZtZLD z6o1wE#6x<{<%%rqICpC%{LIO%WB-2A_2!0m>lJY!I&QK>5yT@NnO9}XsSqlaqN+d& z>A*iiZu^=ZzM_RY*OLu;k)>I8?$fh8sE5CBk+0^3BL5lRER}tEshHzs#aHtCGU33H zp-9F*cd?X(@8Z3*2!tfwIg`>W-?{GyFRDU=$G&~#`3`8RSkzs$(9|?wh3Z3Q4 zk|3&{a3tjS2RuXh0GFpUn9YTAK90`@x#6(ilZ`}zo}my8c(Q?9z~kovxllHk4UBMp zLU#Ta8rZ!Y_Sh9J(myd7OGeWZ$xJF8O~*2O;;D2pK01|-PmE^{Ozeq~zSLwQo;DL= ztX$u8G%*#+q+-c0N2to1g75HLK z*+W&DhVS=&c>3la9Dgk1Cawh8l*cq9Nl{5(;xL`1K5UjRAt>tZd6{fqwsxcNbC3LM z%Xi!lmfXahAibO<&O+Y{k~mhR3RJvVtGGzTf{#V*PLTCQsUUE)h|N+-AOUcE8ohjs zUOq-IAH%|@=k+DbGP~!?vog7{ws{Qgq%;R4!l%PcIsqEPb#!5zmH=9>1&I@^cI7C!pSYmM2AitLLP@hS_aU-Q zP{u38B7#nkwFo>ZNvcZ1buh+EjXMa-0czxanGCJ8Nmy2I7P*EHz-kk+rWZNL8xt!f zYhqa?9m$D|ejs$Eep?gcyT% zyDnUTS@G6UviR$J-~TwP#>J`RI!h0m*|hGx)~q=y%X9(9*ZlF{-tw*I>7yOQ6(^kv z41FLM%vEzZyt@IPP*Zf%d13&=PF7GY2z z1#W_O>~VbWoG$}iSrRm)Ii8xP^4JWOmoR9YbQgKCpw7%=y&GNkW6-CPHPg}L*#2np zf%E6jGv>V!0B)Fr(Id@e6qU*<01h-ofi%phP@+_}gLcPBPg&+U0jHws*d1z)ldE;I z@w}w%H>KkTVz5jcn9LlEr}s}xr85|lIW#eji$VwR{0du+Q1p&t9FIiW3`9!C_Dv+1%9E%%pVa{6-D8AfQ&Dg<-NI+jx1RNC#aZ$BXk1WBtdo;c;KAx@uSff-* z8CaSDNON=71tF|~>SOz2CbAoHFiHfGQ!|JR;^$}R23ga#vX~nHb#uflzZgbUsU&d5 zqIKiLsZams*tUQhCK#;S4eLptIn8L}HQx@Kvy!owVEL4Lea%JT91HL#{;)KQ?a#X|8K+I>fX>L^fcr-dr6D^>Pz6sV2@)lr}thsGfD;{(TBtvI%9 zi{8djprU2Og--fcgC@<_CHfi%L>c)wV`0dG#nzQE(l>KYX%UaJ+ox<=zU==ZzP&5UeiP z)1!yRH+)vhQZ4QH+mC5wUU*px!QPcUU_8ej-M$y!t7Q$ii+|9{tbJDtv5W9QD>QtJ zZ+NXKdlj^q;h_C}?)tOhp5LA6Tl41WLEq^wAcBb8BBmNxGD~R}>6wGQrd*=(5lk== z&$ivhIUk~-$Y40&$qx^Yc!t2L*^|u&gPvi3ekhy``YHUNfj@&E^trX&db67`Z0?#G z2P0=zpuRP8YsM=Atn|Ke^V{%3*5DsJi|oH;fBnz4tZJuXhXO69z$gCRauAmpK-;I- zV3y|ngAvY?5Bqt~P$0~CB9y0|FrN!@k!&!}g}~r`+j^i2TUOnqm4=DNwT{7>)8Kkr z!wa1ltj88o(lK8&`$k;#bwI*5TfdaG9m;T477a zwVr#kH7sYpvu3iK#>|ZK4zddy11A5mO`sNQM%tLN-!?}0Szc6~;Zf(ZmovH7yX$q? zCiiv?kM;UeK??w5D)l!*F|#rC?uI2Wmp7+kh zviw#h;L2=VKc$tq~qB|H^xVZRO2~Am3`o z7y2}OQOYN0@asFwi}!8j41T+zUb~!$t*XI)or33+lIYQu&Lvtd!&Y>ep?}%r?{@7( z+6AZa%wIcs8ZM96v9fns5_0ItPyzpv7Ol@fgLvD{22I&KCl{IJ)zCrsK(D0iG3)^~ ztE~RtXyMRZED>}GWcx|@YbN_=Az#&0(6F6bE*xYq^glcq4>Sv^XAExL!vW&9Ae1sJ zCE>iJ3qE%=sD$Be!*4U>MpHC1^89&b^yhg?qtvbwkfV=9!HfA|p{?+V;jV{SJfqZG z@Y+oFx|A|Z;S7oy;PBf&*g{1+(?#>_(R!THV~iz$hJYx4gq@% zgmoLt%_*rOb9|Dg;7T@y1{Hd)Hv5^STrkXMb3sp#4pGk#pZ9}-P$=Z#L)mOF1g+z; zd1%=?@MAWZb=!@&HDimp5+}2A&33x6u7jf2N$==_%1_Ed37nkH^2`yXsQ?*RF3~oJ zux>E_e)7Pk$DSO0Y4MBOCVv0GhdZ8Pm@YA^HFL8!{QzR51_5KD1HZZ45$5{WJ63(2 zl+gr16?o%Z)a@o~50O3)U^y!1#;C~g;Px|jh@dtJqxMqpNDSY-# NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 10, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 11, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 12, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 13, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 14, + "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "SessionWindow", + "gap" : "PT5S", + "partition_key_indices" : [ 0 ] + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "needRetraction" : false, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[SESSION(time_col=[rowtime], gap=[5 s], partition keys=[name])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 15, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 16, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 12, + "target" : 13, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time-two-phase/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..c0715f55b5f88c201420f7a964cae5b98ea05bcc GIT binary patch literal 23215 zcmeGkZHybmb?m!ba!KGWAyAT#_E0W?Cb8G{*=M_{1ip)tm^gP=+jj?{!mQV8XLI(h zv%9{#)k!u7axfoqT7+|-JOs3-n@A;^XAQ)N!(xb5JI})hry*i$m$y(-22emw}14XEuX#k z$-6&(al4+jfayjie7A(Huk88O>Ylalo?f`{Lh4lFRY<~|3obI?&*h>4E)wvCeZxMM z_ea7JU%tTdp+LkRjB;VfwqDN;4;l<4E+oxGHl(Gh%<(Cn>|vWmR6!^__xH;=jhs zvNBVUN|2~9Wj~W04$K6jS-#q06qR;f7Bm{ zMEyfSAdXSkS-v6(qUs4oBiu0S$%R5;zL59%ha$sFAW{ep`wIn+&sPY0f}v2rlgkA; zkB4GnaG$4aayi_Dwdp#XVdXiB9=@XiKigeO=L+fdf5ug9p$}+k|55&hv4O;a#<3+ z>WKeAlg)(oEj`+OF%OYE{2ts|7t@*F}fOgMw ze zpZdeL@4w1E*+pCl(yhQ80PKRXY7V=157Z~r65n*07=-73?9n=JKsdTU@bkRF$wEaF z>WvnePOGvY7Ez`}=oH9-t?-U5j_=*`72qpP0zz7osVP<-J<7_91T;Zs$}uqhBSfr(&tmgR#_u7cX9F3!+37^);P_-Fi!s?F<722Vbb-t-lXAqO zHyo9CWYPv;QYwC6G8s$N7_?Sr5HIFw+@K3{-iAo=Wmhd@7bhYEnU)whA`}y%YUdtV zROg@iDEhA~wt3!E`&ZFxBT)lW}u54d4~LD+7XPx{SaM(ZEX{Nvfk$-U=CFjIvh z{4ynBpeD9=K(Q~;Vsv6pN-(Jb3LXV$YUi;ZJ(s=crSoEU9%5B;qt*v8Gt6gtX6%L- z(;yk);244=Hf-%XFy-J9=|G@m_)JRB>(9J@_Vq`%Ux1cg~oV*ZEJ8VZ}F?{DS7?i@dz?&(A#j$*I#30G%*EBY|VIOqen+s+Gs+ zar$aBD1VCchEwWv^aF*XefL`K)1uFS~R>Im7fS=ZHxS=Ty( z>+D?SsAnH>uH<;VGFzlG^HPsFFFDOl)qY()x2{ig^|vD(@oF{ur{;EFRJSd9sr}PZhb}U<45*-NTY%cWY%tqH>0*wOItEnB9^!KCe~tkaw%fgz z!9d4=s(G|FG06@#iMN;}HB)VS%#zoR0TmV32>xNnX^&Ht^KbN3fj{rOrai&yf`(rG zX?(+Htt_jho%--;t<3M<(4OG$N&`60(O~fzu9Y?LF1)RkS@U=8iA{t7z0fci-|$*j z_8Op>)$_P3=!iJ(O7sC&Ape07`DjaICe+Q?X$Ygene&*FpOYP%0c5807b!Mj36zj~EtDze1! z3Y_aIW;J(m4^72`V1}aTB8_LCM%NoHpbbZs4ZKX~Ik;ZbFT*VC*wwF$U42<@@hNuEQD``NzKSccCr>s$ zaK2Wl-L%k9)dCk@M!UL!o@u6rZUzr1nd$-_h;MBHWDzD93Nc*EehOWw*fbpWYj(I2 zouA=sBRebbb6z;mJ_S#$()6DmZ}&D);%epa)Vay&mnMF3*Lm(7TvULEzenl{H}|); zw^$n1B_&>!#1qe*e*RVOTwDVfYz?3tmR==jNH_{=(SW;2d(4p6&UVIqb?mfj=;+3# zZ5%ZOLdqAR2Fiw~0^<+GWfJe$!YCPIOPiM(Zwl>6CtTopIPX z#&KDnA#4patggec2}o|HU)8VBn|BuItY)9H@k763#3JTy@%KFZ_KaJC~4H zuX&b6Z+(2w(X+Kl$61(Na$#nX=k^;eoc^LOMn3=jK3C%2g3G%E;|r(pb>u=F!_mdo zU)rPQT>Y8%X4}l06+si#4CVST{JG2=PUm+z|22A>){5%R@D>_Nz9Ea7TCT86Kd^?5smt^eWPYoJvWy`yl= zC~Q$}M>i|OMG$bUg3p~zfY3VI>a_uKqpy=0dGR78*z=yop*^P|M}Msj@Sz1Pv>9IU zx#wXTM>+H!ytb0PA%_fKD-8tzhZh@%&?Rw4t0>m!V04}|DMo8xtBJuAGZJ`W1l>d# zsawYx0t$D*ax?TUTkmM?bwb8XN>$*EofEg4tT{sZLGj>NIX}vZ91r&u=8q7x0$|jB z7H-&u?>^&O5C!F^H6~rn!3}H}V}MCll?vPohfzHR2`=u1m)@!fHc{A$&%>3JCT(ch zYN(a#)nLr5jM00r3|{Dn!A0QOH-9TkNE=uCV)18A|8dQ;cW-)`UQM_bXe_hz);X1g z>Tl)aB$;RUd7cB)kt8xTK>KUOjLQpE((q!#O>hOLYBoZQUI(d?gY~Ey)!It;ARKG2 zg9N;QTegPkn1FP9Yjk@X&}|4)D_2|gj>g8u#}cElB$&1J79Oa<1Ok8^i!LE NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { "id" : 2, @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,8 +134,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { "id" : 3, "type" : "stream-exec-watermark-assigner_1", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -169,10 +187,6 @@ "id" : 4, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "VARCHAR(2147483647)" @@ -180,6 +194,10 @@ "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 3, @@ -200,14 +218,14 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -217,62 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, c, a, rowtime])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { "id" : 5, - "type" : "stream-exec-local-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$3", - "internalName" : "$SUM$1", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S", - "offset" : "PT15S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 3, - "isRowtime" : true - }, - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `$slice_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[b], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s], offset=[15 s])], select=[b, COUNT(c) AS count$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])" - }, { - "id" : 6, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -282,49 +247,77 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `$slice_end` BIGINT>", - "description" : "Exchange(distribution=[hash[b]])" + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, - "type" : "stream-exec-global-window-aggregate_1", + "id" : 6, + "type" : "stream-exec-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" }, "grouping" : [ 0 ], "aggCalls" : [ { - "name" : "EXPR$2", + "name" : "EXPR$3", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" }, { - "name" : "EXPR$3", + "name" : "EXPR$4", "internalName" : "$SUM$1", - "argList" : [ 2 ], + "argList" : [ 1 ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], "windowing" : { - "strategy" : "SliceAttached", + "strategy" : "TimeAttribute", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S", - "offset" : "PT15S" + "type" : "SessionWindow", + "gap" : "PT5S", + "partition_key_indices" : [ 0 ] }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 3, + "timeAttributeIndex" : 3, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -362,39 +355,23 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "localAggInputRowType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s], offset=[15 s])], select=[b, COUNT(count$0) AS EXPR$2, SUM(sum$1) AS EXPR$3, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[SESSION(time_col=[rowtime], gap=[5 s], partition keys=[name])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 7, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 4, "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", "inputIndex" : 1, @@ -403,6 +380,10 @@ "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" } ], "condition" : null, "inputProperties" : [ { @@ -412,10 +393,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT>", - "description" : "Calc(select=[b, window_end, EXPR$2, EXPR$3])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 8, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -426,12 +407,15 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" }, { "name" : "window_end", "dataType" : "TIMESTAMP(3)" @@ -439,15 +423,15 @@ "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -459,8 +443,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2, EXPR$3])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { "source" : 1, @@ -511,12 +495,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 8, - "target" : 9, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-session-partition-event-time/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..b42a43e0f8457ac99cea9f38d2987d7578c33c85 GIT binary patch literal 23219 zcmeGkZHyb$@!5B|-$J6xCn_LMJoN4s)~Y8D56cJs%a`zsUJUDKomh$iYhffpwOxawWw;5P+Mp^^LE#} zd$upW5Ic#oPqN?c`<{98-kY5_Gs*i)9zsYj{4lt*h^+qc$ql!i4Hjo6=Dzu-`4jtI z(DN2C-zbF7may*IXE*#>e&~kBei?Z2$X|Xm2w7Ni5sS2_viRE5k9_dnE!VAi?Va+k zdY^n)FS%Tmh3#i=&5b;JV$0aSpK`sq{@rF(TnLVvY*qyEut(-qndMZ7m0DR%UddA~o%sCMFZPY+?bRR*{U)lsnB*0+3;7KP1mWvlU0gLcC_dTdr-W41ioBQcC#v* zh4=SJU52GB_+IB`E5yd6{fmwsoWObC3RO^LN}2 zRouj#A_JTx&cfUak~mgo6=-;q)^M2>OAJlgl_KlPQc2+0G7d{6ffV$^X!v3%z8H!x zhC;#UWto&&XV-jnRwg$#4v&JJmgWFO7<$~K7kZ0N`*tErj&z9h1ssm@=43Wq3-gNClxNP@sUWKN#5NKl8UQ4N~m&mLbK!-Olqj zU^cvElq~+{zV|;)n{ja}xy~}fW-)DguQgka%QCxw>udhl?{4|l^Xy|i#FZqy3M_p< z7wlDYxV-xSo^V@u(`jND;yt*cb=&}P^Z?-(c!iUNsz%h;SZF$<%7R!znigSEAO>!N zXPj|-?wzj!UTGE#q&=FNX63ONR$fA&Nzzy5#gaNRkL_-7J%G`mN>)$D(_{PM=?Bl9 zJ4cE4M(A+;9IPH`E~luhtU||uwlI)|9TjR+s`jqkNzz}Hc}~EssJ?bX?NM^QO}d_! zjQyr;a(@DjiT#tg1Ig^ZiK%Q3Q*s9<#_>?-0h(Xo$gvlF;wZ%Y;Y36r(-s0pg;G*f?bstrVD1}D?oEzo>m96CYLy%u z&Cp4Ed)Ik;SPji5_9jeV*B4-w2qLHE5Ez8dPr(h+p>3flHvs77idp_JjH^;b;EY4- z#z!)r{?E~^0XJ+g*tQ$alRvy6f~P7OK(?KTd}*R43azpxX7c8a>!k z5*%vK1&;(Yz5UR4pUIu`(sQx50I4eZn6?J7Fq~(4VO)lo(j*((!94_r*mSnL`4TT*^X3yL-#>Od3QZ@Crg4Ddv`&N~FQ(PUg;y2iMWL!d=h4<6O-B`s$pK z?wm_~=DFldJ7xQI_T2h?qO-qUS%{Zv+CK_&adj?Gor%(y%?0yKS10le{!quBLwzg# zOxCH-!esT4gKaZe8biE<&uW%vW3}#YyM1JI=~vv6E?T5pdO>!vhjg!}$p)C%s^I0M zN<|ValD#~u@LpuMEqtl{)dE8g8DBxGz-(I@wY}Ne?951)%7h&ODslmi0JSi>;0RD1 z0jhUd39Km@bp)vNuu!~)bl(x6Is#PNZJ~XLiTgN`R(l0DFbu_sFpj#mCFAd0V-U#gG8>b zc(zgmIs#NpXR(DycDO~lMI@<-YC9vAymADn$hb!F4?|vyj#W>;JWvDtyz!zIgVqHN z{d#P?!)v`RtK}Vg`*E$#3omOi=)2MY%5ya6`n~vEuWR(X_(!eIns>Drrw9XTp(7Y0e}8&#^_!=B%&9LRfC$_o<{D5^O=%bDpM!5rxx&haF~i6_ z({&x^STItI`9m>JaKs<-gbHlX!?Ll6r&!F#B85nv4{$LU;LqR(b8a2CKJ0cBn^#SZ zgOW3?&|I6jGwT&QtPQ+!^V{%1R^uOpMf%^dzfUk7Yuahp!9d3`F!cG3cX5e3==v7R z^ISL*;-a2tAe#4tBEceTv3%ac6@7U>9}5J-ESQme5|!@hvFawBR7^C?I(!Yc!OcDm z&v#>Rb$%pT6T4x01LBim)Sq(Ax*sHK{THliG|tr32f7gc9@G%@yaHuq+>UYBfh z@7D5Irymt;sYGpa;jn;1;;zI!iQT$sYWAMV1X)GxP9f9X;sog=m(uI#VCuH37brq> zG=S0eW2MM*1lSixfW0E>OS4OCj={!z6<21D9&K*fy;iAPvNvV5fO};$U0G^8OeM+; zQiI7ZpyhdcA3zpiK5T^;u4ILPo^<>GSk<+%Vnn_lj0o9TfuHk&<@hwj8qoBg+L3!( zFmd&IXx?pN`po1{?>x<&0*?S_Wj)$ZxwT@mea15IT~g*%NnCj5`1hXo&Ly-4L+v$a zZ%YRWCK7BeEgWz!>5Lrm%DFeVuK}HQ4qb&+q}|ZKqMb&w&Lz6^&6921x#nHMZi$&_ z(QgZ)8n)Azt@+L4aWI{pjE^wZX6|j&oejXAoNx4y$K>d=j!-@mGys=&ctE zELL;P+3=w|^RSTl<>Nf0wY~IZ?-*qJG8F4IjaKy8$B6Ep?HxMKhuL{Q%q;ZWdBMfy z;%w&%UN0BJLBL1zHgVZ;d|mNeV`!Av)=OvDoJ)W5-e5a>Pw4B9nR|)Hu+p4Q-@n5IlO_(Hlbn#||tybZqahaum$@uSXeTj6dpVm8n zzWT|w6Y$qgak!W#K3Wa_3@xo@@oh#SjPdR@&~6v~V_??}S5#YrP|MK8gT7Y5>-H9n zP?&9SZGzIM>|`dMJ4XlXd25ko>^u%72J0}uix#lb7I?&X=OZ+YV(2Y+Y^Qir3>nH+ zS{EATY#2h-xNYrru|{u3$4QH76apJ022RXGz=;uF{pZc!f9`(+ug|{l^RL`{_ES$n zI?n4N=(81q#ekn<^V~?0_Z17p$OsqB^D%!oU&sdnYygU_*Na0$L!Bfvoxe>PsU~xL znrFdDX$G(A^-^8-U>XU9eIp|=@M{|OML@eR8r*#?XL485ZZDnBU;6>y(9%TpJJrodx(xlG%hL8rm|`^keF zAAf4}rNu99o%q9pA8vn|Lb^m@(=1J&bkTOS0RUrT1COlSVd_-YJ63<3l<{&~6?o&u z&Fv;@4w6A&SUFZMjIkoegG0%}L4wjFOxnYOuUmK@Fx~}GP-d(tnOYuP1 z7pp&U{M9ul@80|@^&Pnu##mvgTR@eB8{H;ylFT#wJkNp1Q4*PYLwoAg4DXUEY3g}$ zGx$hV%}Iz;f1N7X*Vt7vsIB832EPLhEdegT4O*}P2{>+VecawA NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { - "id" : 2, + "id" : 19, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,10 +134,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { - "id" : 3, + "id" : 20, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "CALL", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -166,19 +184,19 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { - "id" : 4, + "id" : 21, "type" : "stream-exec-calc_1", "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, "type" : "VARCHAR(2147483647)" }, { "kind" : "CALL", @@ -188,7 +206,7 @@ "internalName" : "$HASH_CODE$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 1, "type" : "VARCHAR(2147483647)" } ], "type" : "INT" @@ -218,13 +236,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", - "fieldType" : "INT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "a_int", + "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "$f5", @@ -238,9 +256,9 @@ } } ] }, - "description" : "Calc(select=[a, b, c, MOD(HASH_CODE(c), 1024) AS $f5, rowtime])" + "description" : "Calc(select=[name, a_int, comment, MOD(HASH_CODE(comment), 1024) AS $f5, rowtime])" }, { - "id" : 5, + "id" : 22, "type" : "stream-exec-local-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -264,7 +282,7 @@ "distinct" : false, "approximate" : false, "ignoreNulls" : false, - "type" : "BIGINT" + "type" : "INT" }, { "name" : null, "syntax" : "FUNCTION_STAR", @@ -279,9 +297,8 @@ "windowing" : { "strategy" : "TimeAttribute", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT1H", - "step" : "PT10M" + "type" : "TumblingWindow", + "size" : "PT5S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -291,7 +308,6 @@ "timeAttributeIndex" : 4, "isRowtime" : true }, - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -302,8 +318,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", - "fieldType" : "INT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "$f5", "fieldType" : "INT" @@ -312,7 +328,7 @@ "fieldType" : "BIGINT" }, { "name" : "sum$1", - "fieldType" : "BIGINT" + "fieldType" : "INT" }, { "name" : "count$2", "fieldType" : "BIGINT" @@ -343,9 +359,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "LocalWindowAggregate(groupBy=[a, $f5], window=[CUMULATE(time_col=[rowtime], max_size=[1 h], step=[10 min])], select=[a, $f5, COUNT(*) AS count1$0, SUM(b) AS sum$1, COUNT(distinct$0 c) AS count$2, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])" + "description" : "LocalWindowAggregate(groupBy=[name, $f5], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[name, $f5, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" }, { - "id" : 6, + "id" : 23, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -358,8 +374,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", - "fieldType" : "INT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "$f5", "fieldType" : "INT" @@ -368,7 +384,7 @@ "fieldType" : "BIGINT" }, { "name" : "sum$1", - "fieldType" : "BIGINT" + "fieldType" : "INT" }, { "name" : "count$2", "fieldType" : "BIGINT" @@ -399,9 +415,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Exchange(distribution=[hash[a, $f5]])" + "description" : "Exchange(distribution=[hash[name, $f5]])" }, { - "id" : 7, + "id" : 24, "type" : "stream-exec-global-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -425,7 +441,7 @@ "distinct" : false, "approximate" : false, "ignoreNulls" : false, - "type" : "BIGINT" + "type" : "INT" }, { "name" : null, "syntax" : "FUNCTION_STAR", @@ -440,9 +456,8 @@ "windowing" : { "strategy" : "SliceAttached", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT1H", - "step" : "PT10M" + "type" : "TumblingWindow", + "size" : "PT5S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -479,7 +494,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -490,13 +504,13 @@ "localAggInputRowType" : { "type" : "ROW", "fields" : [ { - "name" : "a", - "fieldType" : "INT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "a_int", + "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "$f5", @@ -510,15 +524,15 @@ } } ] }, - "outputType" : "ROW<`a` INT, `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` BIGINT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[a, $f5], window=[CUMULATE(slice_end=[$slice_end], max_size=[1 h], step=[10 min])], select=[a, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` INT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name, $f5], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[name, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 25, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "INT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 5, @@ -538,7 +552,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 3, - "type" : "BIGINT" + "type" : "INT" }, { "kind" : "INPUT_REF", "inputIndex" : 4, @@ -552,10 +566,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` BIGINT, `$f6` BIGINT NOT NULL>", - "description" : "Calc(select=[a, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" }, { - "id" : 9, + "id" : 26, "type" : "stream-exec-local-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -578,7 +592,7 @@ "distinct" : false, "approximate" : false, "ignoreNulls" : false, - "type" : "BIGINT" + "type" : "INT" }, { "name" : null, "internalName" : "$$SUM0$1", @@ -592,9 +606,8 @@ "windowing" : { "strategy" : "WindowAttached", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT1H", - "step" : "PT10M" + "type" : "TumblingWindow", + "size" : "PT5S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -605,7 +618,6 @@ "windowEnd" : 2, "isRowtime" : true }, - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -613,10 +625,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `sum$0` BIGINT, `sum$1` BIGINT, `sum$2` BIGINT, `$window_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[a], window=[CUMULATE(win_start=[window_start], win_end=[window_end], max_size=[1 h], step=[10 min])], select=[a, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, slice_end('w$) AS $window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "LocalWindowAggregate(groupBy=[name], window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[5 s])], select=[name, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, slice_end('w$) AS $window_end])" }, { - "id" : 10, + "id" : 27, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -626,10 +638,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `sum$0` BIGINT, `sum$1` BIGINT, `sum$2` BIGINT, `$window_end` BIGINT>", - "description" : "Exchange(distribution=[hash[a]])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 11, + "id" : 28, "type" : "stream-exec-global-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -652,7 +664,7 @@ "distinct" : false, "approximate" : false, "ignoreNulls" : false, - "type" : "BIGINT" + "type" : "INT" }, { "name" : null, "internalName" : "$$SUM0$1", @@ -666,9 +678,8 @@ "windowing" : { "strategy" : "WindowAttached", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT1H", - "step" : "PT10M" + "type" : "TumblingWindow", + "size" : "PT5S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -706,7 +717,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -714,66 +724,36 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "localAggInputRowType" : "ROW<`a` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` BIGINT, `$f6` BIGINT NOT NULL>", - "outputType" : "ROW<`a` INT, `$f1` BIGINT NOT NULL, `$f2` BIGINT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[a], window=[CUMULATE(win_end=[$window_end], max_size=[1 h], step=[10 min])], select=[a, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, start('w$) AS window_start, end('w$) AS window_end])" + "localAggInputRowType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f1` BIGINT NOT NULL, `$f2` INT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[TUMBLE(win_end=[$window_end], size=[5 s])], select=[name, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 12, + "id" : 29, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - } ], - "type" : "BIGINT" + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" }, { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "TIMESTAMP(3) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" }, { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 5, - "type" : "TIMESTAMP(3) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" }, { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "type" : "BIGINT" + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" }, { "kind" : "INPUT_REF", "inputIndex" : 2, - "type" : "BIGINT" + "type" : "INT" }, { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "BIGINT NOT NULL" - } ], - "type" : "BIGINT" + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" } ], "condition" : null, "inputProperties" : [ { @@ -783,10 +763,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT, `window_start` TIMESTAMP(3), `window_end` TIMESTAMP(3), `cnt_star` BIGINT, `sum_b` BIGINT, `cnt_distinct_c` BIGINT>", - "description" : "Calc(select=[CAST(a AS BIGINT) AS a, CAST(window_start AS TIMESTAMP(3)) AS window_start, CAST(window_end AS TIMESTAMP(3)) AS window_end, CAST($f1 AS BIGINT) AS cnt_star, $f2 AS sum_b, CAST($f3 AS BIGINT) AS cnt_distinct_c])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f1 AS $f3, $f2 AS $f4, $f3 AS $f5])" }, { - "id" : 13, + "id" : 30, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -797,12 +777,12 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" }, { "name" : "window_start", "dataType" : "TIMESTAMP(3)" @@ -810,23 +790,18 @@ "name" : "window_end", "dataType" : "TIMESTAMP(3)" }, { - "name" : "cnt_star", + "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_b", - "dataType" : "BIGINT" + "name" : "sum_int", + "dataType" : "INT" }, { - "name" : "cnt_distinct_c", + "name" : "distinct_cnt", "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } + "partitionKeys" : [ ] } } }, @@ -838,89 +813,89 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT, `window_start` TIMESTAMP(3), `window_end` TIMESTAMP(3), `cnt_star` BIGINT, `sum_b` BIGINT, `cnt_distinct_c` BIGINT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, window_start, window_end, cnt_star, sum_b, cnt_distinct_c])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, $f3, $f4, $f5])" } ], "edges" : [ { - "source" : 1, - "target" : 2, + "source" : 18, + "target" : 19, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 3, + "source" : 19, + "target" : 20, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 20, + "target" : 21, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 21, + "target" : 22, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 22, + "target" : 23, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 23, + "target" : 24, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 24, + "target" : 25, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 25, + "target" : 26, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 9, - "target" : 10, + "source" : 26, + "target" : 27, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 10, - "target" : 11, + "source" : 27, + "target" : 28, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 11, - "target" : 12, + "source" : 28, + "target" : 29, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 12, - "target" : 13, + "source" : 29, + "target" : 30, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..ac5b8d0d0db861601dd5e03eb79dda5426c3ce5e GIT binary patch literal 29223 zcmeHQdyE^!86W#Dm&YBs3xqZagr?1fM-spH{AgNH-^EEBoV%0lOCVI3^?L1W&R#pa z>wEo?UPRTp0sCw zRTLCoIKVIwChQM~{C+Rv4f}lokIYW+m9iizE`|%_{dpHN5F8lbgIVuDIO1X0K*S$n ze6Aeh^SPL8DD27x{5}_(%lp_6>&=I`5LED&@YiLow^nTjSvN8oOGeWp$xJF8O~*1@ z z_CA048~C;u|G_g$|J(l-7zI9i?*eP^oaTP6G{A{NDAxc zk?1`B53)#>h_^`@C~akTv0M;1w&>o?N<;ftDGi<)!#JG85hqq2_1K z$GMj*6XfA)v52ipkR@1lvRqaa5==n#JnD?#p)-PqlE6c&=h3R?F=UJBwoFwV2PML^ z)U-KiwdRqF34vBE-_K64?jkD|+%d6Q8#@?SQOM%m(d5vMX!0v3PMpxlxw)k>UEBE{lnuGd}lk9Cm!$!7GDUu)-=5W51r-}K88}e_i zWk63E29@c{l;m9iwe(^z1=dZr3p^mkM+ohHKC^2C-W>a0N%a7qVH+Y(yVx8P$<~C zzP4KG8jNoT{@{_yq;CJy=404Yy^10s$S#=$;9QY-%D^ae4y*h_*wU4OwF(ne6q~YoL8OGphUj#$BU-?o&I<+Z0MlHz?LJg-S)eq>;DX!Sv$$137j`T0qd*7Aj8#CmY2SE{LZ!A z-+A>2iUd?OE~v#HnUAe9P5m~Hm-e6t^F7b5GR?CY5wuKo6G#)fR#WatAq;z{+e7{A z?~2<){md-MaDUqM)GRe8>wg?RR@HWMwZUF@(B!QD3HlAw2HEfw{A(pT1Mzfiy=D-o zTdmRT;2&1xYW)LU-0W?+h$UJ|$B7d(C)X>cCTXEr)jjkMR-Zm|oAe6!uF}HD$3(Ym zl5Xp0bWKgdeR6-RMJycxBSF~CS|5N^OZ7To1NG<5-wu8F-10;FSKof^JuczJ3ixBu zvly<|0pATz-#1@+^3tb&^}s~`mM3aE6qt6@8~#eLp{$GH z8O9YJ;BqcL5@d3Ih6_isNYgjrk6M0O)*IW_iUnw2YBUi~*G)rXG@UhyFEuP}ny))q zNn)qngVPE>DhVY)5hnO3des-B{T~--S~)_MjL+j=J$l#j?rSz(cc=efn;tp*8iY|M zz&LuB8so$J)vF4>aqWM3@ZEopeD~*l8}HL%&SJDCNv3_z^UlG+EFb2*J_an7*`SNbaamV3?+d&9Y(DGt z^Y9w+5waN@3>@5?8tq`>b48EaCh*0atiu)WJ#zf=pB%a$uun&VEYD+@9waNIPldJ- zp7taFLqbgY9EU;HG{_1Sp8+dKl`Tn!lXNCXH*M?S^e|iE71C>HC?Ho`ZX~^0_-I)M zAyO64mN~6q(;k z@;)q{E-hO#AcXu8s?yqqwwMXhMe8mfAP0%#+}lN%7uB;PIkE?q=U3A8JMyo4s9Y>U zmqAy`{bOROoD=c_wK4_1{%`LF-(zomtB+Ezvj{#q?KYo0gL=lR2g&SnH+^sg&Bnng z)rQmM0z0Q|7u~mW~6YpN{St&FqP%cZ`gsGw74qJ2H%#O$Q(=r&x0=Me8_ff{f*@ z03s!0+s6{oWQ{<}Gy?Hrj{5aGFy>WQD86j0MeN{2B;c7KR^~QqNKUPQ#d5Wj5ojT; z$u0|n2y+RGH7l7~5j{G>lEKDcWFKvAfOf}f>VxZ$`iMDQydU#+0<3O80M+qv?4J*X zXSO^#2x}n_(-`dvT7`9J`%S;Eg=C+mqZ=)K;dl}N7%-{w$UwWV-2cOeGbh}1d!Q4r zd6;}IRmUJk2K#j_G7dxZu?WU|P=;XHmc4x~hV1y5aH!zffIqPqjsCTUub>U5cJk1@ ze1VrP{_C-aJ~(tZ3{o+!BdeHln5GF);ziW-=siMT^=>u8gzYrv>C!fc&W>bjRYDp} z$h5(Z#swG9F^E%?dhL_f3$sV&XI|>6&r8}Uvb&ugm-?j6BYjfcekRJr!W^KdV~u8^ zH!V!!>H48<3#sjqovJaZz8PCTbl27PLw7efS-~~h`kQ-c8=STQ)U$n}-3Cx8K5hf3`{sa<##;y|iedw(8mNK? zj}Tn40aP16g;PT|fLgm&tLfp1Q@%&W22iytatxU6sjkP265K}8=mxrWIAWn8X*Pgr z1E}=gL;EnfVXM;O>Ma{UJ(KxH+hDf=R2x7A+vECOsWyOWxb4kAmyR}_zs7ZHSX-1TZR?Y50Z*Prggv4hi*S;5tJ9=LH@*R)=B?Csu zji%)1*uBt!0Fw*Dp#Ybc8;HQ^0C>xKy`0PI_xf2M?`7ZuX=sLhFSMbX?N^%uHU-XC z3e5ZQ6<;>%4Ta#uRUn`Dx|kpb2e5*Pr1-3zC z{d$fw!Sf6ko;ccWWYMwEZy~n1_w1t4^UA&LF{fF$ci!5A!5$1Ic&CG$4>zoK*s$c8 z!IB#NPWv>LZB=}R2j&(g0UND~+Xb7RKNhm{eHWF!=)U`win!<1_j})XaOSPx>exr% zj}YuEh8wV0{M4WIg|%DX{^HJO4nDC&+I-tbyTJcsNi;zdZZPfEjk@7?+_-W2qjT<| z+UuxYsUY1rxxV;z5fO2pyzg{;yeP(&1oYo9aJ*r3F z6u>la8j}*wK|o&kS|LJ8{|!D&e(dlcmOOO*$|sxz&Sx)!I+j@at3V1FXgr$Cl_lQ8 zPw^a_YA=hPT1MMy$@CN43b8!hy%O#rQ0ko!rDx(5vZJvmt5zH9-4C|C4LJcV;IMO` zfe2V{cdg!Ti+bz)*UJ6*ONXMvBg65bXaerk)OvVbEl;QbkYmmd2wJdWEdlm|WowLe SLIc93aMCR`mKu!>Z~Z?K1=~FU literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/plan/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/plan/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split.json new file mode 100644 index 00000000000..1bef5bc9e7d --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/plan/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split.json @@ -0,0 +1,908 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 48, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "dataType" : "INT" + }, { + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "TO_TIMESTAMP(`ts`)" + } + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ { + "rowtimeAttribute" : "rowtime", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "serializableString" : "`rowtime` - INTERVAL '1' SECOND" + } + } ] + }, + "partitionKeys" : [ ] + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 49, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 50, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 51, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$MOD$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$HASH_CODE$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1024, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, MOD(HASH_CODE(comment), 1024) AS $f5, rowtime])" + }, { + "id" : 52, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 3 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "TumblingWindow", + "size" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 4, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "LocalWindowAggregate(groupBy=[name, $f5], window=[TUMBLE(time_col=[rowtime], size=[5 s], offset=[1 s])], select=[name, $f5, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" + }, { + "id" : 53, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "count1$0", + "fieldType" : "BIGINT" + }, { + "name" : "sum$1", + "fieldType" : "INT" + }, { + "name" : "count$2", + "fieldType" : "BIGINT" + }, { + "name" : "distinct$0", + "fieldType" : { + "type" : "RAW", + "class" : "org.apache.flink.table.api.dataview.MapView", + "externalDataType" : { + "logicalType" : { + "type" : "STRUCTURED_TYPE", + "implementationClass" : "org.apache.flink.table.api.dataview.MapView", + "attributes" : [ { + "name" : "map", + "attributeType" : "MAP" + } ] + }, + "fields" : [ { + "name" : "map", + "keyClass" : { + "conversionClass" : "org.apache.flink.table.data.StringData" + } + } ] + } + } + }, { + "name" : "$slice_end", + "fieldType" : "BIGINT" + } ] + }, + "description" : "Exchange(distribution=[hash[name, $f5]])" + }, { + "id" : 54, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0, 1 ], + "aggCalls" : [ { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "SliceAttached", + "window" : { + "type" : "TumblingWindow", + "size" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "sliceEnd" : 6, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "$f5", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f5` INT, `$f2` BIGINT NOT NULL, `$f3` INT, `$f4` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name, $f5], window=[TUMBLE(slice_end=[$slice_end], size=[5 s], offset=[1 s])], select=[name, $f5, COUNT(count1$0) AS $f2, SUM(sum$1) AS $f3, COUNT(distinct$0 count$2) AS $f4, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 55, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f5, $f2 AS $f4, $f3 AS $f5_0, $f4 AS $f6])" + }, { + "id" : 56, + "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "TumblingWindow", + "size" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : 1, + "windowEnd" : 2, + "isRowtime" : true + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "LocalWindowAggregate(groupBy=[name], window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[5 s], offset=[1 s])], select=[name, $SUM0($f4) AS sum$0, SUM($f5_0) AS sum$1, $SUM0($f6) AS sum$2, slice_end('w$) AS $window_end])" + }, { + "id" : 57, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `sum$0` BIGINT, `sum$1` INT, `sum$2` BIGINT, `$window_end` BIGINT>", + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 58, + "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 4 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : null, + "internalName" : "$SUM$1", + "argList" : [ 5 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : null, + "internalName" : "$$SUM0$1", + "argList" : [ 6 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "WindowAttached", + "window" : { + "type" : "TumblingWindow", + "size" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "windowStart" : -1, + "windowEnd" : 4, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "localAggInputRowType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f5` INT, `$f4` BIGINT NOT NULL, `$f5_0` INT, `$f6` BIGINT NOT NULL>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `$f1` BIGINT NOT NULL, `$f2` INT, `$f3` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[TUMBLE(win_end=[$window_end], size=[5 s], offset=[1 s])], select=[name, $SUM0(sum$0) AS $f1, SUM(sum$1) AS $f2, $SUM0(sum$2) AS $f3, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 59, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, $f1 AS $f3, $f2 AS $f4, $f3 AS $f5])" + }, { + "id" : 60, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `$f3` BIGINT NOT NULL, `$f4` INT, `$f5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, $f3, $f4, $f5])" + } ], + "edges" : [ { + "source" : 48, + "target" : 49, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 49, + "target" : 50, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 50, + "target" : 51, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 51, + "target" : 52, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 52, + "target" : 53, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 53, + "target" : 54, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 54, + "target" : 55, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 55, + "target" : 56, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 56, + "target" : 57, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 57, + "target" : 58, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 58, + "target" : 59, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 59, + "target" : 60, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset-distinct-split/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..4d05dfb492d4808fba68119afe95f96b61b89969 GIT binary patch literal 29223 zcmeHQ4U8OB9e=xby?)YOp-Mjh3%w%H&VGN|5Zt?N+b!F>b9dXqVOZYH&R(b8o!!pN zeYB>v4MqqA5QPLuw2cx%u_#GXVv0gc6f_!0M2KKg6^&N;5EVi3qyGPIW_D+8XX*BO zcTjF7*?lu_-uK7<{on8X-ka1dLmVNb9e#MZG=t3F{@l}jcRcXgUA;>qFI+9%->ttp zfbR`QF7)Lhb4RHv$Ym<;!WYKN<1X^$qF53>4~N43cr+IG2O_a>z!wh(y$T%>sufXExd0z8 z(f}v;LcUle5{7%h8!toy(U3pP6=PwZ3;BEk7mtL)oZ$CIsm~V|!h8V|_y>QPwbZ$$ewj3+A{$^MST>jt;%ka4DUQr%rF=HR6 zUBnh*(PA;|_i?d!q{xLru>waUfdJ==7Q?}K!50beenM8Ed;}4T;xN}B4pll>55$1BoQ-}(e+fNCVALkGfpJBkG?@>DKhaji36pr;we>zM4pyB+o;^LgUUIu)EkV; zO&3aYr8rZPYAphctF@XiCAVG^=Whw7dUMH*F#0dq{$I!r>OXEW(4-6- zn|)T5MQLbU%eS+%H2&U(@SAHHP*bK(WqGoMIg6~$W1p1siV7N?w<=e7K~W%e`B70S zR7N@WR)GvMg@k3+UJ3eiL-G_=7HAp5V(L((D#%o=$ciV25!RDuNzes~4rQD7E03$K%2wu90 zy#_UvdQQ8p=zANqG`qIaEe<)lzCK&(=uT~f_U}xTaOyWv$>Fptb89Itl zF+l{P$W7d7GK(+Ky?a%%IE`ba7oh=fPl+nZx-^-m)bMCq=mMshUJcA<_yq&Ay%YG5$dYO5-abVu zBj?Y|<>rR^X$pnX(QULeEQ~{ipfB54Tjp^ZWI{~k?jp%1@iu8xUDh^c34Td$VoNgH zpXiyUWv>egqN0ituO@gt?=zZ~jXu@3CD9MBn`u*xc$95RW{YyrHS0ahl6$RiU8cfy zlWk23w-M+|j=?IaL`~hdtI^6XK7G7wCRbDtzKSQIR>~re4Fjr?s<_kD&y7ij>(ks*2N<2~szdOww6>b7clNr0l(VZ2 zvu7AK$m*xyuNCJZaA#=iHA0}lT4TY%A134~;{i=v>}@%UIl52R>C-GE?xZD4`rARx zJopz(KD)+e=_J@L)!&i#ac-L>gX?HEEm^{K@<6Lm%Mw&2##&zt@ z@65hpQ~%I1LQ)TsTC1T&U=ebAyp$faAl( zLOA9x24g&hl>HfgjK*o3Zp>RNbk5Ffe>#<`^E{KxpFJcmH_TI-Pa>%-(>E7qR*@BnwiCeFIY5kUe zEdLCB^Tn6(14?S?pi=X?IotF<8%QzGc`-PC;!K$b}-s z2uGvAL1?29e~}IbiiIL}9EW8Lh2UIH2{~K9j=R zHseW9)4j53OuldU)@xmFoc!UIcmCCL_j@ZoYl}1*Zgq3?vGq^2r+wBR z2@90^LNP874~Dsr;16+eS_pB)c-&6|LB3dwL^_ z#b9i`{rJ&K9y@pl=%21MSy;q(#w$f7of^9r8M@vy(CCns`ypq7EpJ$1(I{%L5nOAQ zq|Hs*(_}u&YxkaBS{78Y$dpka@2$IuEYjciR}@%!sYoih=q3}_xo>c%z^g!ZZr=s_ zT2->rhPZ~LIxoa~J-7gu3#)oiMx@J3k*1+r3WZHn*@4kBc1>%0s3Cl0>*;yJ?dWOJ z!P2hmCi{u&!$+i^meu#ROy723C|$`YOxS)sJ(W@kstl@9>DnrlD+RGAvW@o8uK%-W zwg33*cXzVVYcGL~QFd$396&wbW!>b!Pj7zrG8T-BSINgsHC%V5RqsXhklC^x_OG#n zKfm;=&(g28;h_TU3Y_Qy{AZ9=qbQv_AV1;8)@`$IPj3@lU~a%3U2hz1(C`a_!pmY+ zYt$E-+O)=##5OHKr@#_e0r%K_@VizU>=Xu!Z!r{7Cb@BII+3Xr&^*0> z1gXI6x?Je;lNc%9TvKz{#!Fa$=R(@E=2<7R)gqX!443mFi=;K#Wiw5(#xPr-KGdRE zV35r6m>G2JXW0$1>R85Xkcv_7F%zXcW7uwB7MnyC8;9R$QQI=hS| zVI>}a!Je-JZx^%PjfWk;2~D8;!=Nf)1KM`w&WDfWPkR_cx*h1rF!+2->x1YRj_}YO zV>d(_o7Wf)$`Fj(wzjXsH=Ew)JtS~47@!!9UjDU=FJ>8MM}I3rsP!aXzXy-XtYCaCJE)%i}Ng{-qgwKR>v$; z_I2@#JKoOMvw5axTjwO`-W@J!OjEPUe*WxB9*+$>04iz$ z4uD$d-s1qMlYOYC1E4Z|+yPK`i~}Lf2eC2~#Q{+DDLm2<5L|KqR0lwXgOeNpwf1F! zrsv0;^-gVR%gAb%Hik2XFblL z0F48E>K|@!0Mv;pR5w9-5fl1=($D10Hs!sX{OH6CG&KLwG&B9evR`- z3_t&TM)Pa9H4X(ap6%it?-$WryznRzyo_hKt)d)I$=)FQ@PMX zC0+oMV_|^cp_gd2fx_qW;CYepEJ~uX!)%is9t7GJ+8DuPhm|Uj$v)0B={VS3-hu zNL8|_F)F)Uo73F|Sa?H7P!>QA3^o)3rrT3Xx5p;kMtf@I$o;uJiQc~6R8Jxe$8hR3 oys{Q1BmjzI+&2epnqtiZ)`D$r%xOZvZ7wiFvRkwL$=>z<2U>OJ*8l(j literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/plan/window-aggregate-tumble-event-time-two-phase-with-offset.json similarity index 65% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/plan/window-aggregate-tumble-event-time-two-phase-with-offset.json index fb8c4a6400f..a1b4c49fbbe 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/plan/window-aggregate-tumble-event-time-two-phase-with-offset.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { - "id" : 1, + "id" : 39, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" }, { - "name" : "c", + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,18 +85,24 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { - "id" : 2, + "id" : 40, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,10 +134,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { - "id" : 3, + "id" : 41, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "CALL", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -166,19 +184,19 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { - "id" : 4, + "id" : 42, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" }, { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 1, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", @@ -200,13 +218,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -217,9 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, a, c, rowtime])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { - "id" : 5, + "id" : 43, "type" : "stream-exec-local-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -254,23 +272,13 @@ "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$6", - "catalogName" : "`default_catalog`.`default_database`.`concat_distinct_agg`", - "class" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatDistinctAggFunction", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "VARCHAR(2147483647)" } ], "windowing" : { "strategy" : "TimeAttribute", "window" : { "type" : "TumblingWindow", "size" : "PT5S", - "offset" : "PT5S" + "offset" : "PT1S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", @@ -280,7 +288,6 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -291,8 +298,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "count1$0", "fieldType" : "BIGINT" @@ -302,47 +309,6 @@ }, { "name" : "count$2", "fieldType" : "BIGINT" - }, { - "name" : "concat_distinct_agg$3", - "fieldType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatAcc", - "attributes" : [ { - "name" : "list", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.ListView", - "externalDataType" : { - "logicalType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.ListView", - "attributes" : [ { - "name" : "list", - "attributeType" : "ARRAY" - } ] - }, - "fields" : [ { - "name" : "list", - "conversionClass" : "java.util.List" - } ] - } - } - }, { - "name" : "map", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.MapView", - "externalDataType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.MapView", - "attributes" : [ { - "name" : "map", - "attributeType" : "MAP" - } ] - } - } - } ] - } }, { "name" : "distinct$0", "fieldType" : { @@ -370,9 +336,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s], offset=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, COUNT(distinct$0 c) AS count$2, concat_distinct_agg(c) AS concat_distinct_agg$3, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])" + "description" : "LocalWindowAggregate(groupBy=[name], window=[TUMBLE(time_col=[rowtime], size=[5 s], offset=[1 s])], select=[name, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" }, { - "id" : 6, + "id" : 44, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -385,8 +351,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "count1$0", "fieldType" : "BIGINT" @@ -396,47 +362,6 @@ }, { "name" : "count$2", "fieldType" : "BIGINT" - }, { - "name" : "concat_distinct_agg$3", - "fieldType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatAcc", - "attributes" : [ { - "name" : "list", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.ListView", - "externalDataType" : { - "logicalType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.ListView", - "attributes" : [ { - "name" : "list", - "attributeType" : "ARRAY" - } ] - }, - "fields" : [ { - "name" : "list", - "conversionClass" : "java.util.List" - } ] - } - } - }, { - "name" : "map", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.MapView", - "externalDataType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.MapView", - "attributes" : [ { - "name" : "map", - "attributeType" : "MAP" - } ] - } - } - } ] - } }, { "name" : "distinct$0", "fieldType" : { @@ -464,9 +389,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Exchange(distribution=[hash[b]])" + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, + "id" : 45, "type" : "stream-exec-global-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -501,30 +426,20 @@ "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$6", - "catalogName" : "`default_catalog`.`default_database`.`concat_distinct_agg`", - "class" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatDistinctAggFunction", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "VARCHAR(2147483647)" } ], "windowing" : { "strategy" : "SliceAttached", "window" : { "type" : "TumblingWindow", "size" : "PT5S", - "offset" : "PT5S" + "offset" : "PT1S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 6, + "sliceEnd" : 5, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -554,7 +469,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -565,13 +479,13 @@ "localAggInputRowType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -582,22 +496,22 @@ } } ] }, - "outputType" : "ROW<`b` BIGINT, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s], offset=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, concat_distinct_agg(concat_distinct_agg$3) AS EXPR$6, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[TUMBLE(slice_end=[$slice_end], size=[5 s], offset=[1 s])], select=[name, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 46, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 5, + "inputIndex" : 4, "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 6, + "inputIndex" : 5, "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", @@ -611,10 +525,6 @@ "kind" : "INPUT_REF", "inputIndex" : 3, "type" : "BIGINT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "VARCHAR(2147483647)" } ], "condition" : null, "inputProperties" : [ { @@ -624,10 +534,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>", - "description" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 47, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -638,12 +548,12 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" }, { "name" : "window_start", "dataType" : "TIMESTAMP(3)" @@ -654,21 +564,15 @@ "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" }, { "name" : "distinct_cnt", "dataType" : "BIGINT" - }, { - "name" : "concat_distinct", - "dataType" : "VARCHAR(2147483647)" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -680,61 +584,61 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { - "source" : 1, - "target" : 2, + "source" : 39, + "target" : 40, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 3, + "source" : 40, + "target" : 41, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 41, + "target" : 42, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 42, + "target" : 43, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 43, + "target" : 44, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 44, + "target" : 45, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 45, + "target" : 46, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 46, + "target" : 47, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase-with-offset/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..c9dde6d17f82f6f0957e0345817adff909308e52 GIT binary patch literal 22637 zcmeGkZHya7b!=ZQcbBwxX#?~tRis=(NaOYQolSle-<^{S8=IwhgdRN{_2Y7s zL^w5`NGDU_R3tqXO{Nmjk;znaJf1!{J{BPZ$%$ArReKR(^#-QGvB^j}8A(LLvFPDQ z0&?wSswC$*TBiK8z~u@s2%JhY1%Z<*k}4JjPCZ^0DympexX7F;$Pz6i1zDs8abA!I z8?%xUEi2O{6<|BaRe!?Kvz>j1tW#uiOAAbhLE-j_LLsY+QI)3P`R`Ahz3K6j52x+K z79&^Z@SX8WQOQ8;2%VuWY?dn~D(ap&nQUJcx>5O=)9-Emj{TmZo!DcfpD#%>F!W+c z8ZA%-0B_O&7pRnXv8+8YvY}AQi##o0vsB`Z0Unn|FBhYii_y!)dg0Q`x?;6Dd*;eB zGP$9#d92%s(kze&FNd9U0XEjxk>hb%1Z@4Ljvl1tJ$I0P{ryBq5mjioO0K=b)adcd z5rJ2s)m=wLDXWsLO`3~%U4ZvCC;=s}F6*ESq}Q8}#(*J&?0%|DBl@{A4CC&`o-l^d zhEXQ~2k?<#TGxn;jbk@!yK;=2AT}5=cUyo=l&k@>A0~aG60Z~rh&o1k5P70hQdKh4 zq{mpO^#EZ#=o&d86aQ+Pgmv|1k!uKnUR{l>$sq@E5vfu%zbvc7kr+P&G~B@oA^&ot zr9uIQ42Dt}oRo^Cte6v7r+}~j)4$#G6db_wt86)9(FcxkJQ8UW5GfJaHyH~jss!rO2_#5amN#g_lAlDR zc(bco(7{VcKv!D?91-(TNv)?IS<>h3k?6i?JXJ-oUa1t*ur&jc_SUWyF{}shk$sU` zXE$bHmWUFsrm-{FKM(7!k2URjmUA6+y*6W(-weZQsVMTsrgg)YlArq5{5G#078rzW zhy7%rw#;br!$Yr~s&!tcKSPEc-&CU)46l$EGvbA``j|b=ZjE6jWb>i_!hmWLS*iu^Hp)=2^x>$QM5ueREiZ?(r{o%>8eRyR4^wveR> z#7pF?W|?)O)?IDGM?sgJ;+CLjk#E9+;-Z7B9j1wfKGCws@ugz1B$+gO8L9{zirXf+ zwEof52M39-0xAgGCZKgsbuc^A(xs|lOM!|~fTcjqjvTiXsFnhiTUG;G(~MdQRJK_t zoT1&f6sVR0weD-7iu+2HJuC&Pr9ib5sK!TQkooa~&s?4OY`G?S8%u$TmJus`>Aw;* zX_nrc8G~2~RLx4Le&28Yb@s~C0a^-F(>KJ`>i;YSDtvASja=36Y_$Zm6sVfbVhfe* zV2gZ{N>WqRc1A6E`6y6Pa1G)gUQ|m?zWwlT`zt`7@;NOSwt5>ydM$0;B z)cNN#T3w@T?=p1_yi4oi5aH#}lB!8DJT~B6IIrRBc~?t~x<=cLXLDTx??=49I`uHI z?f8dqu5Wzw#o4ExPyQzJ4rak`0e)uc;-hhWcOdojSkD9Gz{?ALKYsA^6Og}BmBpQJ z{p5V`n*-l_?WJGQ!O)}4sxY%~8|j^eleAo<@=*keKfIn?YXfedXNU$u0qP8e_#hZ? zd*O~B_BwMpniGPMS z-i8~p9{=E4VE-NayXlRNfwll!6zDhwwmjZ(6dy8zwO(O8?o4QihH_5o_Tmcb3pz8w zT*x`ZhXgNm4}1KMU@3w1X$oRC32;* zAXBv@D;ypu-VflYv*sFa)4+9}dpjfKZLOOT!i*rP)?{en zwq`Bm&Aiq=w57Z=lVO9t4bXfZHvznCWZ<^Q?U6Cvr8af@M1-tk{<84WPMYryos>Yj zEZb4t=k@|spO)n^`mC&$c$VV*;wau%#eE67#8(=8)7S9@IzQjsBfU?ldf+z|qd+os zX*Ryh>6vN5UC0eJ`4m0o+dJHtK0>gS64uJg*pR`I2pv^mW{N{KVT)rp5qYa@xIy1Oe8NSb9o=>|C)@mY8ISz&=;B_l$ zs`0GddTPMbtj(~x-s>SpOiX?0gg9yK)qTl_FWIpSe{wa~8h!LJVQXc3Ta0z+TJg|j z;^)q*ATE`VK$r1+xfo##K3GhQOHRI9llVG@-Amnk>CC>j*8A*rb^B~u0x45daO;ci z1*tHssjh1oExhA>iUo~N%pG|h-V9V**jTJ-M-00aqc~!z>j>I;ZCMF)k6s;{Zd`A^5_z=yz{D@3 z-YjnMrA^=e-}LWZe)F}F-xDqdu>m!HF_~{JHt}&i@o%F;2rGEvo2h#$W@P41EPe_2iGTz@*%0^>#wW@7Ag) z7+*i^cG7d041j>aQ#m_IC0>A_gxSLc&-R$LmqNHyc8(hh*Mv{yS%^DXYa42(nr^^)H(a%%E@Paq4sJCXbs)BzryU|B zjX14X{mgwY^*r*q%|B!D2m7FnMan|UsU*-CdXF#3f=ieacu=R7Bv%#C-fA@?LZM2U zMlak9aW>W3AcR@WN0sbvMAfX<=5`N)?|K6ypaq1|@-+|v+s#$m%{6H^BO&~?6Th-| zG#np~M@PdkP)+M2ytN7w8UW;25>B=WD^@GuC^YSjxlI^-trsFWB`1>;k@(pE0c@%n Awg3PC literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/plan/window-aggregate-tumble-event-time-two-phase.json similarity index 65% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/plan/window-aggregate-tumble-event-time-two-phase.json index ef95c7b98eb..af76ff748a0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/plan/window-aggregate-tumble-event-time-two-phase.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { - "id" : 1, + "id" : 9, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" }, { - "name" : "c", + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,18 +85,24 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { - "id" : 2, + "id" : 10, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,10 +134,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { - "id" : 3, + "id" : 11, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "CALL", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -166,19 +184,19 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { - "id" : 4, + "id" : 12, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" }, { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 1, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", @@ -200,13 +218,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -217,9 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, a, c, rowtime])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { - "id" : 5, + "id" : 13, "type" : "stream-exec-local-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -254,16 +272,6 @@ "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$6", - "catalogName" : "`default_catalog`.`default_database`.`concat_distinct_agg`", - "class" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatDistinctAggFunction", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "VARCHAR(2147483647)" } ], "windowing" : { "strategy" : "TimeAttribute", @@ -279,7 +287,6 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -290,8 +297,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "count1$0", "fieldType" : "BIGINT" @@ -301,47 +308,6 @@ }, { "name" : "count$2", "fieldType" : "BIGINT" - }, { - "name" : "concat_distinct_agg$3", - "fieldType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatAcc", - "attributes" : [ { - "name" : "list", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.ListView", - "externalDataType" : { - "logicalType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.ListView", - "attributes" : [ { - "name" : "list", - "attributeType" : "ARRAY" - } ] - }, - "fields" : [ { - "name" : "list", - "conversionClass" : "java.util.List" - } ] - } - } - }, { - "name" : "map", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.MapView", - "externalDataType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.MapView", - "attributes" : [ { - "name" : "map", - "attributeType" : "MAP" - } ] - } - } - } ] - } }, { "name" : "distinct$0", "fieldType" : { @@ -369,9 +335,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, COUNT(distinct$0 c) AS count$2, concat_distinct_agg(c) AS concat_distinct_agg$3, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])" + "description" : "LocalWindowAggregate(groupBy=[name], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[name, COUNT(*) AS count1$0, SUM(a_int) AS sum$1, COUNT(distinct$0 comment) AS count$2, DISTINCT(comment) AS distinct$0, slice_end('w$) AS $slice_end])" }, { - "id" : 6, + "id" : 14, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -384,8 +350,8 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "count1$0", "fieldType" : "BIGINT" @@ -395,47 +361,6 @@ }, { "name" : "count$2", "fieldType" : "BIGINT" - }, { - "name" : "concat_distinct_agg$3", - "fieldType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatAcc", - "attributes" : [ { - "name" : "list", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.ListView", - "externalDataType" : { - "logicalType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.ListView", - "attributes" : [ { - "name" : "list", - "attributeType" : "ARRAY" - } ] - }, - "fields" : [ { - "name" : "list", - "conversionClass" : "java.util.List" - } ] - } - } - }, { - "name" : "map", - "attributeType" : { - "type" : "RAW", - "class" : "org.apache.flink.table.api.dataview.MapView", - "externalDataType" : { - "type" : "STRUCTURED_TYPE", - "implementationClass" : "org.apache.flink.table.api.dataview.MapView", - "attributes" : [ { - "name" : "map", - "attributeType" : "MAP" - } ] - } - } - } ] - } }, { "name" : "distinct$0", "fieldType" : { @@ -463,9 +388,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Exchange(distribution=[hash[b]])" + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, + "id" : 15, "type" : "stream-exec-global-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" @@ -500,16 +425,6 @@ "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$6", - "catalogName" : "`default_catalog`.`default_database`.`concat_distinct_agg`", - "class" : "org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions$ConcatDistinctAggFunction", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "VARCHAR(2147483647)" } ], "windowing" : { "strategy" : "SliceAttached", @@ -522,7 +437,7 @@ "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 6, + "sliceEnd" : 5, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -552,7 +467,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -563,13 +477,13 @@ "localAggInputRowType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "c", + "name" : "comment", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -580,22 +494,22 @@ } } ] }, - "outputType" : "ROW<`b` BIGINT, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, concat_distinct_agg(concat_distinct_agg$3) AS EXPR$6, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "GlobalWindowAggregate(groupBy=[name], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[name, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, COUNT(distinct$0 count$2) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 16, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 5, + "inputIndex" : 4, "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 6, + "inputIndex" : 5, "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", @@ -609,10 +523,6 @@ "kind" : "INPUT_REF", "inputIndex" : 3, "type" : "BIGINT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "VARCHAR(2147483647)" } ], "condition" : null, "inputProperties" : [ { @@ -622,10 +532,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>", - "description" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 17, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -636,12 +546,12 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" }, { "name" : "window_start", "dataType" : "TIMESTAMP(3)" @@ -652,21 +562,15 @@ "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" }, { "name" : "distinct_cnt", "dataType" : "BIGINT" - }, { - "name" : "concat_distinct", - "dataType" : "VARCHAR(2147483647)" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -678,61 +582,61 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { - "source" : 1, - "target" : 2, + "source" : 9, + "target" : 10, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 3, + "source" : 10, + "target" : 11, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 11, + "target" : 12, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 12, + "target" : 13, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 13, + "target" : 14, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 14, + "target" : 15, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 15, + "target" : 16, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 16, + "target" : 17, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-two-phase/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..2f5b6241ed613653923aad576a70a3ea3f0fc825 GIT binary patch literal 22637 zcmeHPdyE^!86W#Dms}FKO9(9_1f7&ic*Sp@A3Y@SU7W<=_)fO(QUb!P*K6PA>|JMf zeRnxka4Bko6bYzWRH~L1si-I@v`Bz>NELrjHR>y%KKO%DQ66e4L?zT#<d%zQK7eDlrBH^2FI6L*zdgph9d;pI>Rvg)G`?(m*?=XcND{OX^? zk8XVZaV>2D)AdXaq;innSw`Wc3L{Npf}Z%SgM6#Vm$|QaW*Es+6w3upl06llCl~Th z2ctQkcmBXoXu9Y>5U2{JQrL@E>3}Ee*yh6`R#a3~Va`ywG1%LG|hFdqrKB7tJi#pVlKDDMvsF@-$v z_jCC5nAz62c94Ow@pw9x9ZTmj*;qE7+mXm*(}|IZY+@{x+cUN!PF75$;GPonPsWlH z@mwaJPQ;RleepEJI?1>wmE25~VW&8EvCIq8@W8uSu~HEQw{oz`)fB!gyYr09xs#$$ z%5W0Tl=*p18Z-l+Ai3ZK*uGp z`N@t~@flaJ+%s$;5bziC(XcB}2!vh12SbAfvFP)MyeuJCPk~rqhQ--1 z+cX<0Ouo#yrJA7d70xgrrp4JEjKUbE!={);Gos_zKgbGMBHk8t;AAVi%VLRVnX-G5 zkw&K&DGQNWuQ{B=ksv)~PADl;bM2OsRLe7$9 zQ5OUcT@XA}1RlzsM`O=pC>H64=c|$5m6I4nlw@}nD{N?( ziJ?nQqA}w%2*K7~TYFn5OJ|I{0n>=!W;j8y2LoL$=v-+}yh7F`ac1_Qk?k2RGqOCKB&+2*uw!XP0V?ECQ6ah$ps0`s zYiO7p!x!n~(3Otuosq~5EoKG8(+aRvoZ-1y_a3G?38CKm5YAO$Pv~wzTc^XlT0_#$ zj=y-z)9kUFlWv(tnuuFNc;+6?BjJaf^1RFqSQ%O`NOSL1K`*gT!0Z z+d`=Z%{pc(enTp@C!QIPjV=>@d`W?q6<%PK7|Z6ox@uY2Q(b#vahzhxi#&!WQM zaNKxb=DJVjxA@g@ytM_7Dy2^I%Lo3o;fKz9D`1`_Ngpc;GjQ<1ivnF4tQU(l8VrP0 zD#U-*kP*N|Fkxn(zGe2u=xTYJ8zKe`Gm8 zXU+UJFRq<26_$IQy~4E4<^-@ZRe)QYbe9BS*hAeO>gWG_$sX$G=8}QuWLu9cP``}9 z=is4V8{;Ie-W&pm4tzESpQn$oT#(I2;P2D21tDFx&g+gext z$Di2(QD-9A^uqS9uk2a<*5QRWf0{m&IIfd`-;gh#kNVk&-y8OZybR}ygd^TUk>Q5? z5nnLMh9T=(EjtAC()rSysMck~n#6KxjsZWZ44#Q^=BVCn`&}v1#o%Q}t)YlZkA@%PLOktZ{s)X>-$h=gCMC zbI?6U+w#&&e@(-KX4M_|c#ofz8t!pq;Ee&W^jPyhPQn*$F*+L_em zpYEOqPu{xenrk0^0*)_?$xrqq=o-g!|mbab~s@;CJ>7H z^RDO+sASORExMv%pU=g4gTX>H7z{+hFif^%dqEpp3!@VXd7)TkJ9(~Lkah6k-%lO8 z>d8Yt0wmLsB$pO3O;3^)(ys#52sV3?fCwSNe1U@+8=JhyrPCLm#Ci>7l&K2gu#Tap9c_ zEQ=x8l^)v*o9%6MH;(%29u>=F=rZU^d2m9ghy}jLQ|~L#>;Ls_^*#Ik{ryyW-DQZ; zWw-sz1vCo2a)d1W=FShVro}i|g?!1-!;Nfe_g>a0nJzXszUB`7e)IQ^Gmmtkr&+fQ zUcLb3!(26slXDNWXQ)}bEgT*+ofw30CywZba&&>>7dV-f_^PVZ>kT@sLVl>z0t^b2 zz$UoI;e+?xb5%G?q)EWY`2hj3+)EsHW6*DQE~Co35h`4(p6lbTSW=*3e_Ikj&n=*+ zRLx_SfaLJZI*8Q#JtreKxjdJhwNI-90vu&0$P# z-&hK*lrF$Y&XRMiMe8_vNQw2W10kj3yC#ycbX`EJH34x#fyNCwFy}Q`Dc)RP&)CHZ zD8O^UpUiF6aG81qE9F`x$J0vMo4YJd=*>0gC)f2V*6Mgk4jY4+eYCj&+8tNYm@A;^ zjS;g{^d8H00!VJa^VIS2=m(EQ=7*miQSm&?b~R;T0DU4U12+NLm+9z6pG5H52RH&2 zRSp$sa@+o&9L=3}Q?S1qkak#nE?37OW(L1bEi(>7jIl_@a&Qj8x-EPACQRA=De1uA z`2aJq8cqK7hOeUyr~aeRy$m9D!Leq2r`TnW8jdD!iylkI`fFR_|6b z=+{Z-oF47C=lslU?aq*2EH@pad$1mtinQ)f=2KR+&Zi=kl~Kt(IS22cwl2W@qw*l0tMIfYc z`%O_48$i_%RD2T#!6h3&wEtCB``6|R&e_z1{P_<7-7%<(l-H#b(a2rXx zn2|IaH>clWu>n;3)o{a6rNt*{YykCK)*JiPaQN2KIS5dHH>L3{BpX0o3RHWs1E{!9 zjcA)y!l*Zg$PZ5R)c{Jp?`LYLx#*JBT1I?FK)dR+`pX15P5;X(t)5qR<=3BqzD}#x zyO$0hunnZk3_#~aFW zI?1Yiq#s}amXQjhjKFf>7g^Xx@Re_j+R4C2SmC)(f93^Vo-)Q{YI*SV!Wg}nOiiwW z{~1R06h(L~0d9J00(|ZQcG?ASeQofD4jz`TU~0|qG_rd110AFRhbJlt2LT^8Upqp` z=wE+D@<$KRvY;q1o*tEB#;I869$@ufPA~_e7i09)|IK9Z~gU- z#!_Rc#AqxDFQIEaJWwwa7yxlB`gE;@D%J_G7c5(2 NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] + }, + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", + "inputProperties" : [ ] + }, { + "id" : 32, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" + }, { + "id" : 33, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "1000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" + }, { + "id" : 34, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Calc(select=[name, a_int, comment, rowtime])" + }, { + "id" : 35, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" + }, { + "id" : 36, + "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, + "grouping" : [ 0 ], + "aggCalls" : [ { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$4", + "internalName" : "$SUM$1", + "argList" : [ 1 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" + } ], + "windowing" : { + "strategy" : "TimeAttribute", + "window" : { + "type" : "TumblingWindow", + "size" : "PT5S", + "offset" : "PT1S" + }, + "timeAttributeType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + }, + "timeAttributeIndex" : 3, + "isRowtime" : true + }, + "namedWindowProperties" : [ { + "name" : "window_start", + "property" : { + "kind" : "WindowStart", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + }, { + "name" : "window_end", + "property" : { + "kind" : "WindowEnd", + "reference" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } + } + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[TUMBLE(time_col=[rowtime], size=[5 s], offset=[1 s])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" + }, { + "id" : 37, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + }, { + "id" : 38, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "window_end", + "dataType" : "TIMESTAMP(3)" + }, { + "name" : "cnt", + "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" + } ], + "edges" : [ { + "source" : 31, + "target" : 32, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 32, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 33, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 34, + "target" : 35, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 35, + "target" : 36, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 36, + "target" : 37, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-with-offset/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time-with-offset/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..ef6ed34683a05a97c672ef18be0cc1773c1e63a0 GIT binary patch literal 22590 zcmeHPZ;Tto6(9R9ms~F7E|9hmax_vdA%)oM|9^UIsqfB7435vTeU}mnX1!kfHfOJ$ z-Syoit>{IO2vsDg3jIKrRKuP1UKm4gf}vI;L%S!o_4^vLHO{ZhicEM71qNS$ z-+-BKjco_nF`bTQV!7!|KAVf>;`ymWHkV0E%;XZ&sr`^A>U={HDzdx4%Dg*S6U$j% z64;7xoR`MTuUV0;%X2jaa_xg3aWsZs-Tp2t|z`$pf*>w=P_`K(-$G5Teyp{BVbo+Z!&cNP<{i(RNmWrsi8mcIB}LNO@TS{eh`V zHIA2M$SJ=lh{f8Xi^j^p5MBN#W7bRVSKC;X6^Uo7kQZHiYjs{?m6{~Gb68-jhnX0< z?k}j$I1NIud#~MlJ9L)bXn8wKBZ6PxMa3Qrbh)5&r9G8jol|*l2E&lH46x-gE&_Nt z$jDvsyW>-ub(Xt34F(yt!XVN~tXGF#Y9RfZ2Ew$hK&qBw?zR&`IUy=JlLCS(ba(oT zVUef80C70UCsywmm`LnTq;i`4_I4#dTX?9^q0$DDi|>zT?A`~8J&q^Tn6*LCmDanO z-j`t)%|q9jBCpyCM&Egk0;4Sxryvps!;$7F)A!XEddG(F<3%N zlVP$h#|(%&PD zF?9*GhN-wEyfM?8$*Ag$YH(j0ryW5Bc5#-iH24+PDU4-{U=!!6IY@a?Wxq8QI~dQVW0PwHAYWG$WJM4;CB|`ikKVN`^r^msu{1y9ml&m@TgMiG%r8$&#galo~(j;mL$VmOjo%bBQ2vl)uA`c~8y$M=&C@s~psMkyGXBU~fX^VzcVYwiqr{V9Tk_8}M*Up=+G*}Q`#`_*MNd3nRvfD0}_24A< zpump8PS~L(gF3kbw*6>D-ZE80FG^3=R*|8)pVX54T)Y2Qz3&aL`3`-_7mOCYMXxIwD8LRq z@AbK&1-{@ac|0Y~8wy7KQE30YSYgQF7Lq)-GAHRnTYw_utEGS^UD&&TH3^?a>;Rw$u0Ce`x!+yO& zc32p#SS|zQ-UTTj<>tDUm0svghC-7tEcxOAR-VJ+sb<~XwgBXhX0ui|bm~Zw0m{34 zlsrHjS09MFCM$1WX8I6VC%dWng8kP$S*ui_$)G9au^F*iD+(onx~o9H{@?H}@AH3q zc!Wx?zXBn;?Dk$fgI2~jO_1eZ-S@#QG#dw}kk1-gxRp$8-Wysu$0Uh8hU07L#BWC* ze}(;FAG+rB%itFY*eA?Yi#R(EKz)Mkz1w1kSnCsGaNma`y45-QpyL;LnUjRN+NrNM z^l23%!9Fd*pg;-S49|$};B)^{9X4NR6fkmSP!r7eI>!Te8!>z1(9XLJ3f!b_grXTC zDNwMlD+!>h3s6++=CR>Skiojda{>ZV&9NJ>7`rId6+TMMJn48|GV^|OE&+&UE_N`T zKa|KFn4ZbyF(iL@I)#fu9~hRG$T^mxH5}d5!Sc3%kTUW8Gs#${DWHv-fOxS;!^Rw# z@>VPqA8uS%iFrE#RWtw~>i9VOk0&C>$Inlw00<_#g|e^%oyaHycL3Sv z=;%f#F7R^(`wbRV9y`$N?juh=oxk9w=x9Gcld$-FMjeBg7#!iW#5fEw#3CB=!8ruW zwruS?Fy_EV!~=s@0{y{aH2F6xzJ*qtI^03?@?~DS{!eG0_~68;2+WEJ9auuj6r~AM z;YC$?44$Tsdb65Ql3sc<3~2cHm6_SPogqJ6eho%{N==-XIP3LpzP>hFWNp@^f!4ZY zH#=4PwH>XG%dz?6YWB-fF0Rc1x*BI33%%*u6rQdhYLkxL`&Q`+X4twN==s*|{T(d0 zc2@^fbg31GkW|Z=Vyr{B!RY}3ygssaCG}?e?vvw_v=SS(0aVliYyh=5am)r#uk@#J zHh@a;aT`FLTLD5E&2NgL*Z``ApyHeGLmNF=JenS8$h+cr8R6-TAblw1E`m?-q_#L!r?fVAwb=kzjb(w z4WO04TM7UcGBRx@4o45S^;Et4XVSU+Xmgep#i} z)9Sv@`a94!Y4vhf(juCR7kXObrN6^tlU6Tx`E|9t4ezOUJ+02W{%)jYF;hZ5?e$e_ zN&i3ptTBviqzd4Z>{r-rSykBT(xujw-Z zZ4CU=jMYdRH~Ov~oCwOz)8k9slg-Dpgr}1!0#83dfNY#jvf(fp0g#GgrQ#$jay)qL z77r8rf*XVOv2dy?ybtT|f+)yy#*l2I0Dg!VqGywB$aV0J#Grvv4SEwi3^qhKy#jEc zA~X3AX#Day5x(_$8hLx8|D2G&9=QB-34Z6=Lb$yA>SKl~yCfs8h5;4zw3# zRcj;PW3ai?mJ`SVyj}fmLO{OVO}^b0d{Ys^UpGhh4Nk^V)2YN{ED7I~Yc0H^nI|v+ h;#hGMtwmRC5@0J>*2c&aBFKZ_>Xe literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/plan/window-aggregate-tumble-event-time.json similarity index 70% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/plan/window-aggregate-tumble-event-time.json index e5a3a7f6460..6b6b9ca1152 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/plan/window-aggregate-tumble-event-time.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { "id" : 1, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" }, { - "name" : "c", + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,15 +85,21 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 5 ], [ 6 ], [ 0 ] ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])", + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[a_int, comment, name, ts], metadata=[]]], fields=[a_int, comment, name, ts])", "inputProperties" : [ ] }, { "id" : 2, @@ -93,7 +111,7 @@ }, { "kind" : "INPUT_REF", "inputIndex" : 1, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 2, @@ -103,7 +121,7 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -116,8 +134,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime])" + "outputType" : "ROW<`a_int` INT, `comment` VARCHAR(2147483647), `name` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", + "description" : "Calc(select=[a_int, comment, name, TO_TIMESTAMP(ts) AS rowtime])" }, { "id" : 3, "type" : "stream-exec-watermark-assigner_1", @@ -147,13 +165,13 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" }, { - "name" : "b", - "fieldType" : "BIGINT" + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -169,10 +187,6 @@ "id" : 4, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "VARCHAR(2147483647)" @@ -180,6 +194,10 @@ "kind" : "INPUT_REF", "inputIndex" : 0, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 3, @@ -200,14 +218,14 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { - "name" : "a", + "name" : "a_int", "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -217,61 +235,9 @@ } } ] }, - "description" : "Calc(select=[b, c, a, rowtime])" + "description" : "Calc(select=[name, a_int, comment, rowtime])" }, { "id" : 5, - "type" : "stream-exec-local-window-aggregate_1", - "configuration" : { - "table.local-time-zone" : "default" - }, - "grouping" : [ 0 ], - "aggCalls" : [ { - "name" : "EXPR$2", - "syntax" : "FUNCTION_STAR", - "internalName" : "$COUNT$1", - "argList" : [ 1 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "BIGINT NOT NULL" - }, { - "name" : "EXPR$3", - "internalName" : "$SUM$1", - "argList" : [ 2 ], - "filterArg" : -1, - "distinct" : false, - "approximate" : false, - "ignoreNulls" : false, - "type" : "INT" - } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S" - }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - }, - "timeAttributeIndex" : 3, - "isRowtime" : true - }, - "needRetraction" : false, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `$slice_end` BIGINT>", - "description" : "LocalWindowAggregate(groupBy=[b], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s])], select=[b, COUNT(c) AS count$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])" - }, { - "id" : 6, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -281,48 +247,76 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `count$0` BIGINT, `sum$1` INT, `$slice_end` BIGINT>", - "description" : "Exchange(distribution=[hash[b]])" + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[name]])" }, { - "id" : 7, - "type" : "stream-exec-global-window-aggregate_1", + "id" : 6, + "type" : "stream-exec-window-aggregate_1", "configuration" : { "table.local-time-zone" : "default" }, "grouping" : [ 0 ], "aggCalls" : [ { - "name" : "EXPR$2", + "name" : "EXPR$3", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" }, { - "name" : "EXPR$3", + "name" : "EXPR$4", "internalName" : "$SUM$1", - "argList" : [ 2 ], + "argList" : [ 1 ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "INT" + }, { + "name" : "EXPR$5", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], "windowing" : { - "strategy" : "SliceAttached", + "strategy" : "TimeAttribute", "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S" + "type" : "TumblingWindow", + "size" : "PT5S" }, "timeAttributeType" : { "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", "precision" : 3, "kind" : "ROWTIME" }, - "sliceEnd" : 3, + "timeAttributeIndex" : 3, "isRowtime" : true }, "namedWindowProperties" : [ { @@ -352,7 +346,6 @@ } } } ], - "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -360,39 +353,23 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "localAggInputRowType" : { - "type" : "ROW", - "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "outputType" : "ROW<`b` BIGINT, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "GlobalWindowAggregate(groupBy=[b], window=[CUMULATE(slice_end=[$slice_end], max_size=[15 s], step=[5 s])], select=[b, COUNT(count$0) AS EXPR$2, SUM(sum$1) AS EXPR$3, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", + "description" : "WindowAggregate(groupBy=[name], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[name, COUNT(*) AS EXPR$3, SUM(a_int) AS EXPR$4, COUNT(DISTINCT comment) AS EXPR$5, start('w$) AS window_start, end('w$) AS window_end])" }, { - "id" : 8, + "id" : 7, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" + "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", "inputIndex" : 4, "type" : "TIMESTAMP(3) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "TIMESTAMP(3) NOT NULL" }, { "kind" : "INPUT_REF", "inputIndex" : 1, @@ -401,6 +378,10 @@ "kind" : "INPUT_REF", "inputIndex" : 2, "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "BIGINT NOT NULL" } ], "condition" : null, "inputProperties" : [ { @@ -410,10 +391,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT>", - "description" : "Calc(select=[b, window_end, EXPR$2, EXPR$3])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Calc(select=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" }, { - "id" : 9, + "id" : 8, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -424,12 +405,15 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "window_start", + "dataType" : "TIMESTAMP(3)" }, { "name" : "window_end", "dataType" : "TIMESTAMP(3)" @@ -437,15 +421,15 @@ "name" : "cnt", "dataType" : "BIGINT" }, { - "name" : "sum_a", + "name" : "sum_int", "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -457,8 +441,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL, `EXPR$3` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2, EXPR$3])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, window_start, window_end, EXPR$3, EXPR$4, EXPR$5])" } ], "edges" : [ { "source" : 1, @@ -509,12 +493,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 8, - "target" : 9, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-event-time/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..a9734ee75c382079dfec4385a64e2255539ba9d6 GIT binary patch literal 22586 zcmeGkZHyaN@!5B|eHWTab=SMkzD?|1 zXLo%c0ks!ZYAZ#eRD1{orBRhq6;gpHfeMKa31};*s6;9J$d5u>1gZcbs8j(hVdm|w zy*{t+d}-_^&8}qc?%Vff-pu=&H*aPV_ZB>akZ$VwSpLo#7xm1z)UFUDfMxHyhW%S=qx?W%Zce^4kB*#rQtD4(EJb$&_KYS^9#Np%|!yy2I>;4hK9@o^qbS@KDqr zitv%aaFmceNN>+l%)pj$k?rFX@l-4`p30^(u}nO>FOklq5+jqD#Q0eD(D=SM=}%82 z6PenFI4jpb6-!RWv*~y$5lbeH#8Z&!CR4If@N#92pQhe?5gNg(ak(P(Diui+OVq2) zmuW>4i>fz1r%^@XifO8dTv0qhm4U{jw8WLwX;}lklLxg>P6 z@bi!SYxCFL50u=*og{s{EX_dEi?TFYNe5iyq=Jtn?M;$(MY$mIToJ3Kkzf+= z_;h~x7{7dsUp_{K&&cab*2?UiE6*t8#>VO~vQzRbun4a%H|Yj!jMkC)F|GvI`YePV z;>vsPCVj^D30W02sJKS1z1vdh@!V0$Yf$U%W1=KzWV=mu5w8#M-U&G%=f<)D%0#-w zf;0&of(i#YbsEvnm7y8;G*ZGkjNXg}1H1qq8K#p)T&x{?Sl!j*rfpdTTY(L{( zcM(!d-tAc)Kn?Jg5wh^Bd)~R3fpPH~xz5tUwQPFxUR#44Qxt9<$JgA+-)#HJ3*2K} z#FZf3Dhz#K7tB?&IJ|p+o^VTiGy22;-1p;%Hh2S_qYFB|KviB5%X+82#?q(Lnj%UC z?9&nq3go~|@Qf{v&)sun;4AwC6=|(cO>xTTG^Z@;pb64bq*6hfo zsI{J4g~`VAqPpLdNgRs9GI3}kdpMCfI6j%lVoLVN_!uqE@Hvs9i5wrAS z7}MmE$eWASjSr_k_Ma0wf^L{#5VjlElm6N?qmPf%|2PvpvG>^#%v5FwZ)R24UhTHG zL$T{wF}AQLWmwbz1;q|Dwd?3Np3YwIvURarfK-iqL?45g8P+o+GY&&cvH6T}a1OyL z+P3!X_~qad-=Tt*=`$(As6X}Q=U#eb$7?Wk-9c8(aK#FRIo3k_Sf!Ms%7bUWxNY4R z{&1>Byup8l3<=*<;}AQ_G6ij{CsumHdA*D`&f-Yl7PnTw9YjuX~*sAO5#HwqZ!3}nGJF(t*)DDoiYDn6EGRB^kAYvNs87MlpKF8yXTTyDVo<=^S} zw!fv{&9Y|Q&36D*EolZep>D?96IlJfWc3yheRxMdqmr?puYd zZY}O%U(pY$CezukxSwoje^pZ`R{~hz%_Mr|GF3QDR#Y#F`by0P!6M?D4ocvCRyF2z zdhh#iq3O6T+R||C@4f92^0wEF2s%S0%^3_f4rR_%Zs$+-flcM@84T-;Wq>ObnDZ|z z1TNBlXZ)`CKEv5HbJs+itYV(5OxCbp9oi{@bURj^hBxd*DkUA8V)Q0iDf1ka_@z;a zuc-H>xJ6#f;5ELAFLEbN*!zmFRja=7wqg`Wras;JmboJ{9kma8Lrp$KU-s5EHi|Qo0j!5*kOU10UYr)` zwwttP!M$=$BJOEmr{1_WK9=0<6grA7vd)LGY?ExW9jy25j5opCq*D#l_pv(go5y0{ zj6D$>Z3x2Ap2po^dL_qrKI=ETmW@2sbtpCgpWVb>nrhr@Hy;`>G;1TQj(Q^)h^13s zKEX$t`&(c3fkk#L(LcR4rx&C3@y6!L)|MD&)3xlT%hI3QuY$N-g7sX%LmJbd z#(KY%!8HExfNb}2pnFHb)fY|_b__yp)5MCnE?i)EcD_+Zh>5C-1&vRvguDS?`l~f; z%w|i7X}4l#M=W)OppntGy>|Q{nqpU>w9hH#UDR<_xX=L0qF>Q0W5li z!F({lb2&ber~bT@ul0%$qi)<%+`76y+c|WExK^ zja;3L92v+9gIpv)J^9d3&J)Tf?lhC(4>g&2tI2o2)rP`eEtk zIYk)dB%VSbKj8?$T{kA}=ODx=y!V;!q9m%*)|7N52cgL@#RQYCsAUM4hDkkn8TubQ z^j0L$IKm;e0I?ctbwj<8?b2wF;j3l!nHZ5a%2SukW`Ko~SSBsfZ{WM}5@ozK4rqpVU5Myx? zHFB^Kl^xaAdJllVcLOBg1%#g(Y+wS`+gq)-*QVZ#h49x(%*Wo**x2}3VlF+%Ay{p?VzmIaf-N=HI$^}JUWk5_o=i`~$M*dn2i6W< literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/plan/window-aggregate-tumble-proc-time.json similarity index 66% rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/plan/window-aggregate-tumble-proc-time.json index a2e0babc152..dd0cc1fdb15 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/plan/window-aggregate-tumble-proc-time.json @@ -1,21 +1,33 @@ { - "flinkVersion" : "", + "flinkVersion" : "1.19", "nodes" : [ { "id" : 1, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "identifier" : "`default_catalog`.`default_database`.`window_source_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "a", + "name" : "ts", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a_int", "dataType" : "INT" }, { - "name" : "b", - "dataType" : "BIGINT" + "name" : "b_double", + "dataType" : "DOUBLE" + }, { + "name" : "c_float", + "dataType" : "FLOAT" + }, { + "name" : "d_bigdec", + "dataType" : "DECIMAL(10, 2)" + }, { + "name" : "comment", + "dataType" : "VARCHAR(2147483647)" }, { - "name" : "c", + "name" : "name", "dataType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", @@ -26,12 +38,12 @@ "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 0, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" }, - "serializableString" : "TO_TIMESTAMP(`c`)" + "serializableString" : "TO_TIMESTAMP(`ts`)" } }, { "name" : "proctime", @@ -60,7 +72,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 7, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -73,24 +85,21 @@ } } ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } }, "abilities" : [ { "type" : "ProjectPushDown", - "projectedFields" : [ [ 1 ], [ 2 ] ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" + "projectedFields" : [ [ 6 ], [ 1 ], [ 5 ], [ 0 ] ], + "producedType" : "ROW<`name` VARCHAR(2147483647), `a_int` INT, `comment` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" }, { "type" : "ReadingMetadata", "metadataKeys" : [ ], - "producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL" + "producedType" : "ROW<`name` VARCHAR(2147483647), `a_int` INT, `comment` VARCHAR(2147483647), `ts` VARCHAR(2147483647)> NOT NULL" } ] }, - "outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, c], metadata=[]]], fields=[b, c])", + "outputType" : "ROW<`name` VARCHAR(2147483647), `a_int` INT, `comment` VARCHAR(2147483647), `ts` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, window_source_t, project=[name, a_int, comment, ts], metadata=[]]], fields=[name, a_int, comment, ts])", "inputProperties" : [ ] }, { "id" : 2, @@ -98,10 +107,6 @@ "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, "type" : "VARCHAR(2147483647)" }, { "kind" : "CALL", @@ -113,12 +118,20 @@ "precision" : 3, "kind" : "PROCTIME" } + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" }, { "kind" : "CALL", "internalName" : "$TO_TIMESTAMP$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 1, + "inputIndex" : 3, "type" : "VARCHAR(2147483647)" } ], "type" : "TIMESTAMP(3)" @@ -134,10 +147,7 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "proctime", @@ -147,12 +157,18 @@ "precision" : 3, "kind" : "PROCTIME" } + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : "TIMESTAMP(3)" } ] }, - "description" : "Calc(select=[b, c, PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])" + "description" : "Calc(select=[name, PROCTIME() AS proctime, a_int, comment, TO_TIMESTAMP(ts) AS rowtime])" }, { "id" : 3, "type" : "stream-exec-watermark-assigner_1", @@ -162,7 +178,7 @@ "internalName" : "$-$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 4, "type" : "TIMESTAMP(3)" }, { "kind" : "LITERAL", @@ -171,7 +187,7 @@ } ], "type" : "TIMESTAMP(3)" }, - "rowtimeFieldIndex" : 3, + "rowtimeFieldIndex" : 4, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -182,10 +198,7 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "proctime", @@ -195,6 +208,12 @@ "precision" : 3, "kind" : "PROCTIME" } + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" }, { "name" : "rowtime", "fieldType" : { @@ -211,20 +230,24 @@ "projection" : [ { "kind" : "INPUT_REF", "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 1, "type" : { "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false, "precision" : 3, "kind" : "PROCTIME" } + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" } ], "condition" : null, "inputProperties" : [ { @@ -237,10 +260,7 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "proctime", @@ -250,9 +270,15 @@ "precision" : 3, "kind" : "PROCTIME" } + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" } ] }, - "description" : "Calc(select=[b, c, proctime])" + "description" : "Calc(select=[name, proctime, a_int, comment])" }, { "id" : 5, "type" : "stream-exec-exchange_1", @@ -267,10 +293,7 @@ "outputType" : { "type" : "ROW", "fields" : [ { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", + "name" : "name", "fieldType" : "VARCHAR(2147483647)" }, { "name" : "proctime", @@ -280,13 +303,21 @@ "precision" : 3, "kind" : "PROCTIME" } + }, { + "name" : "a_int", + "fieldType" : "INT" + }, { + "name" : "comment", + "fieldType" : "VARCHAR(2147483647)" } ] }, - "description" : "Exchange(distribution=[hash[b]])" + "description" : "Exchange(distribution=[hash[name]])" }, { "id" : 6, - "type" : "stream-exec-window-aggregate_1", + "type" : "stream-exec-group-window-aggregate_1", "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", "table.local-time-zone" : "default" }, "grouping" : [ 0 ], @@ -294,58 +325,58 @@ "name" : "EXPR$1", "syntax" : "FUNCTION_STAR", "internalName" : "$COUNT$1", - "argList" : [ 1 ], + "argList" : [ ], "filterArg" : -1, "distinct" : false, "approximate" : false, "ignoreNulls" : false, "type" : "BIGINT NOT NULL" + }, { + "name" : "EXPR$2", + "internalName" : "$SUM$1", + "argList" : [ 2 ], + "filterArg" : -1, + "distinct" : false, + "approximate" : false, + "ignoreNulls" : false, + "type" : "INT" + }, { + "name" : "EXPR$3", + "syntax" : "FUNCTION_STAR", + "internalName" : "$COUNT$1", + "argList" : [ 3 ], + "filterArg" : -1, + "distinct" : true, + "approximate" : false, + "ignoreNulls" : false, + "type" : "BIGINT NOT NULL" } ], - "windowing" : { - "strategy" : "TimeAttribute", - "window" : { - "type" : "CumulativeWindow", - "maxSize" : "PT15S", - "step" : "PT5S" + "window" : { + "kind" : "TUMBLING", + "alias" : { + "name" : "w$", + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } }, - "timeAttributeType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" + "timeField" : { + "fieldName" : "proctime", + "fieldIndex" : 1, + "inputIndex" : 0, + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } }, - "timeAttributeIndex" : 2, - "isRowtime" : false + "isTimeWindow" : true, + "size" : "PT5S" }, - "namedWindowProperties" : [ { - "name" : "window_start", - "property" : { - "kind" : "WindowStart", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - }, { - "name" : "window_end", - "property" : { - "kind" : "WindowEnd", - "reference" : { - "name" : "w$", - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } - } - } ], + "namedWindowProperties" : [ ], "needRetraction" : false, "inputProperties" : [ { "requiredDistribution" : { @@ -354,32 +385,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL>", - "description" : "WindowAggregate(groupBy=[b], window=[CUMULATE(time_col=[proctime], max_size=[15 s], step=[5 s])], select=[b, COUNT(c) AS EXPR$1, start('w$) AS window_start, end('w$) AS window_end])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT, `EXPR$3` BIGINT NOT NULL>", + "description" : "GroupWindowAggregate(groupBy=[name], window=[TumblingGroupWindow('w$, proctime, 5000)], select=[name, COUNT(*) AS EXPR$1, SUM(a_int) AS EXPR$2, COUNT(DISTINCT comment) AS EXPR$3])" }, { "id" : 7, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT NOT NULL" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL>", - "description" : "Calc(select=[b, EXPR$1])" - }, { - "id" : 8, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -390,22 +399,25 @@ }, "dynamicTableSink" : { "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", + "identifier" : "`default_catalog`.`default_database`.`window_sink_t`", "resolvedTable" : { "schema" : { "columns" : [ { - "name" : "b", - "dataType" : "BIGINT" + "name" : "name", + "dataType" : "VARCHAR(2147483647)" }, { "name" : "cnt", "dataType" : "BIGINT" + }, { + "name" : "sum_int", + "dataType" : "INT" + }, { + "name" : "distinct_cnt", + "dataType" : "BIGINT" } ], "watermarkSpecs" : [ ] }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values" - } + "partitionKeys" : [ ] } } }, @@ -417,8 +429,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT, `EXPR$3` BIGINT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.window_sink_t], fields=[name, EXPR$1, EXPR$2, EXPR$3])" } ], "edges" : [ { "source" : 1, @@ -462,12 +474,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-window-aggregate_1/window-aggregate-tumble-proc-time/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..4a6f4cd0827a5256c3ff8ff50c0ca4b4d0d9c779 GIT binary patch literal 18217 zcmeGkZHODkb)@_7=lRag)!1%qOmjAFZS6`rNhifnsyoZE68TO@^0~38<9ahkH&S4jAGX(m*LmO)!);E|jE!QfTOpHVJ8eHH7}~kF=$wG_+9K&_HqPzM0*X zc2Ck-clKSJdKaYKosTzf-p9^(DYf#zF<=(FwG^qpIMzf_;y@ygGG?_c=S8v~FkCFnMM9$ z-1mIJgRYfCi9F6vW0fx{Qjw?{UkUK}Nbr0pmJj&nFN7oKOTi1nRk>Uq@zbk(fzR>h zYO<&g2ZsZpKp+y1j)a4uNFd_(^BSHeRYjC_UnquSMdB;qVlXfgE|g+IkdKuF90~+u zzDQIE`a;4;!551SkN6@b|8OA^4wEP;A+#qB`1LN;c}i9f+B=a>Wa7DrOg@{7=MwoN z$!so@96gasPK@V|PaH|0fowXJ%r!C+tlq$6JarhpSka zB3wxVqH{VfNQ6^svMyE#r_WVMO&2ANOU&v-m9dm1s)!|Vo~T33Nm&_JwJAjh*j=da z&y<;-n}0(+nu-p#X^FO=aYscW6}2N+$2oZa|%n zr!h~_;!~ogAD&gwz9pesmEXDg&%sx{&sMz1n?n79B2NSBMMWNyum*s)8h|A%mwA?S zIE6MzN?8=JMA6bwFa_=K2EBMjFP_nhXT9LfvV5vh=kRQGT1C5>(PQ1tC^JAJdUbhG zH?+q3IyyIwE6`fMt)s_r_3&xbZ|0{JP1FH#9X)c|hV(??91(QDy8FB+7j?9^1-i6d z9@>2hNUB$yV$tr<-2pTJ?;J%7zdG~j9tOrE=xDo*!;NAFdmm{)j;ku3 zqxv;_={LK-{Vsl`3we^LTLaYxa=|PzL)E(*`V;PmZl+8O!Tl)JXp=S|99&Cb2p;h1Ep@nnb-4k<0qj zEQQ!cjygUnmXosXYz{4IrFS&>L~=Y= zZ(yBPtK?xqfkryhxVyHn4!|d#NM!0wY%T(CMOo1Ev@^7~Lmv9SpP{ysrR;Fdf z)2WmKqHyJpFaG-KV;_M$KZ$y#u~Z|VcsAJMwMv1g&)xji?oBWK?n(o;**(?{g}b$Z z4p5JjiMs6%*Kd4!>2ef6CoRwv`53G76^7d(qdrZ3oIP4pLul~ie}G=bG@WZB8qE_c z=mRC1ZC-6kaZIbAbLNzKf5g30wOO;F*s~Txv0bfAa^8zJKb%A~M=g_nD;}sGPU2Yu zvMYV&@_My z`Raf=z6_`^RoQ?#-W)Jj2h+t?wri78d&;#*i=%U{P3qdD+>%CZ!`O6fQZ`v=oV8)% z+N6lu)vitI+N5*~<{`BRY+FsPRd8=sfoqe}4aeo0RIW`5QU``x{m5sz1axgub!e?^ zQhJbZfd0`78@EevlFBYPmSLB9a7>@;WeY~uC3=oQuPw&SI%KAA2NoY@Q-haY>x`TI zzYg~^#KDHo!iV2`{`M!kx3B;3W9b*&uRRA@6!`^O#Oeiowp}0m<3nUf@7m9^o5C@^?X0x~j&}xL8>3h)D ztWr}2k|7ul+_O5?jpMT_T{Oco0`|? zf`qTCia<2*BrNA=z&}cv@v&U$dT1>mJ*c%O-Pf9?Z%tPTt^h1>uoF3@N>r>Xs>bDL zgN>QmF&P^^bKn43cO`J8_a3~u*4&lA#<&p zNNprKgmSM#N60jhbyq!Fcs<^26W`s{{J&I>P3BaA%Vp-H4wD5;5>F+bP8>0Pv~y3V z6R3wdU_++2%?z@ds}##+%=K|v;f|iWpX}}@+fF3FTY62_7Ax_v%iu-M_YmyRBS?6D zzIBMhMy(#w(Q>H;t5@FeaAIEcEF6TV($H8g&=8Ey3?SRyIt+#M0tfKbaOhtX_HhyE0FZUxT>fQp_9=H|XnuKs# z4jvADS)FChXFv1qaLl}U8CIgkM!7i*uV|aYResZsTzNRTtNhNhrnQRSeyEh+T56d) zU(u!onia|l-;NN@v(H~}WA(?hN>3kjkyu-zUN2gI3Jt)BB4D*RhGl_3=zH-LqRV@l zbQHr!IpDqDdKYC;o3f{5YXu04rYRPfY)z{Yxk!_GOA2@w!NZ1{44WNrrcs2i8#HjE zCTmNLGrPezit^TipVpzWMK2G6SnNP2M95lS1EJNgU;beIjjs>>ghjb;1RN`vg@WrS z-2ABmK~V`$W{Chcp(68jKu7D*Y@1d2CT$D3<`w