lsyldliu commented on code in PR #24162:
URL: https://github.com/apache/flink/pull/24162#discussion_r1462995182


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);
+        counterState = getRuntimeContext().getState(counterStateDescriptor);
+
+        MapStateDescriptor<Long, RowData> windowStateDescriptor =
+                new MapStateDescriptor<>("window-table-accs", Types.LONG, 
inputRowType);
+
+        windowState =
+                (InternalMapState<RowData, TimeWindow, Long, RowData>)
+                        getOrCreateKeyedState(windowSerializer, 
windowStateDescriptor);
+
+        windowAggregator = new DummyWindowAggregator();
+        windowAggregator.open(
+                new PerWindowStateDataViewStore(
+                        getKeyedStateBackend(), windowSerializer, 
getRuntimeContext()));
+
+        WindowContextImpl windowContext =
+                new WindowContextImpl(
+                        shiftTimeZone,
+                        internalTimerService,
+                        windowAggregator,
+                        windowAssigner,
+                        triggerContext);
+
+        windowFunction =
+                new MergingWindowProcessFunction<>(
+                        (MergingWindowAssigner<TimeWindow>) windowAssigner,
+                        windowAggregator,
+                        windowSerializer,
+                        0,
+                        new MergingConsumer());
+
+        windowFunction.open(windowContext);
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (windowAggregator != null) {
+            windowAggregator.close();
+        }
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        RowData inputRow = element.getValue();
+        // no matter if order exceeds the Long.MAX_VALUE
+        long order = counterState.value();
+        counterState.update(order + 1);
+
+        long timestamp;
+        if (windowAssigner.isEventTime()) {
+            if (inputRow.isNullAt(rowtimeIndex)) {

Review Comment:
   I think we can add a metrics for the null rowTime



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);

Review Comment:
   Please use `ValueStateDescriptor(String name, TypeSerializer<T> 
typeSerializer)`, and the name should be `session-window-counter`.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);
+        counterState = getRuntimeContext().getState(counterStateDescriptor);
+
+        MapStateDescriptor<Long, RowData> windowStateDescriptor =
+                new MapStateDescriptor<>("window-table-accs", Types.LONG, 
inputRowType);
+
+        windowState =
+                (InternalMapState<RowData, TimeWindow, Long, RowData>)
+                        getOrCreateKeyedState(windowSerializer, 
windowStateDescriptor);
+
+        windowAggregator = new DummyWindowAggregator();
+        windowAggregator.open(
+                new PerWindowStateDataViewStore(
+                        getKeyedStateBackend(), windowSerializer, 
getRuntimeContext()));
+
+        WindowContextImpl windowContext =
+                new WindowContextImpl(
+                        shiftTimeZone,
+                        internalTimerService,
+                        windowAggregator,
+                        windowAssigner,
+                        triggerContext);
+
+        windowFunction =
+                new MergingWindowProcessFunction<>(
+                        (MergingWindowAssigner<TimeWindow>) windowAssigner,
+                        windowAggregator,
+                        windowSerializer,
+                        0,
+                        new MergingConsumer());
+
+        windowFunction.open(windowContext);
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (windowAggregator != null) {
+            windowAggregator.close();
+        }
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        RowData inputRow = element.getValue();
+        // no matter if order exceeds the Long.MAX_VALUE
+        long order = counterState.value();
+        counterState.update(order + 1);

Review Comment:
   If the timestamp is null, the element is dropped, so I think this logic 
should be moved after if check code.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/internal/MergingWindowProcessFunction.java:
##########
@@ -44,6 +45,7 @@ public class MergingWindowProcessFunction<K, W extends Window>
 
     private final MergingWindowAssigner<W> windowAssigner;
     private final TypeSerializer<W> windowSerializer;
+    private final BiConsumerWithException<W, Collection<W>, Exception> 
accMergingConsumer;

Review Comment:
   BiConsumerWihException isn't serializable, so maybe we can't use it directly.



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperatorTest.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.SessionWindowAssigner;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.utils.HandwrittenSelectorUtil;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.time.Duration;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/** Test for {@link UnalignedWindowTableFunctionOperator}. */
+@RunWith(Parameterized.class)
+public class UnalignedWindowTableFunctionOperatorTest extends 
WindowTableFunctionOperatorTestBase {
+
+    public UnalignedWindowTableFunctionOperatorTest(ZoneId shiftTimeZone) {
+        super(shiftTimeZone);
+    }
+
+    @Test
+    public void testEventTimeSessionWindows() throws Exception {

Review Comment:
   We also should test the changelog case?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);
+        counterState = getRuntimeContext().getState(counterStateDescriptor);
+
+        MapStateDescriptor<Long, RowData> windowStateDescriptor =
+                new MapStateDescriptor<>("window-table-accs", Types.LONG, 
inputRowType);
+
+        windowState =
+                (InternalMapState<RowData, TimeWindow, Long, RowData>)
+                        getOrCreateKeyedState(windowSerializer, 
windowStateDescriptor);
+
+        windowAggregator = new DummyWindowAggregator();
+        windowAggregator.open(
+                new PerWindowStateDataViewStore(
+                        getKeyedStateBackend(), windowSerializer, 
getRuntimeContext()));
+
+        WindowContextImpl windowContext =
+                new WindowContextImpl(
+                        shiftTimeZone,
+                        internalTimerService,
+                        windowAggregator,
+                        windowAssigner,
+                        triggerContext);
+
+        windowFunction =
+                new MergingWindowProcessFunction<>(
+                        (MergingWindowAssigner<TimeWindow>) windowAssigner,
+                        windowAggregator,
+                        windowSerializer,
+                        0,
+                        new MergingConsumer());
+
+        windowFunction.open(windowContext);
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (windowAggregator != null) {
+            windowAggregator.close();
+        }
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        RowData inputRow = element.getValue();
+        // no matter if order exceeds the Long.MAX_VALUE
+        long order = counterState.value();

Review Comment:
   The initialized value is 0?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);
+        counterState = getRuntimeContext().getState(counterStateDescriptor);
+
+        MapStateDescriptor<Long, RowData> windowStateDescriptor =
+                new MapStateDescriptor<>("window-table-accs", Types.LONG, 
inputRowType);

Review Comment:
   Please use `MapStateDescriptor(
               String name, TypeSerializer<UK> keySerializer, 
TypeSerializer<UV> valueSerializer)`, the name should be 
`session-window-tvf-acc`



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);

Review Comment:
   `session-window-tvf-timers`



##########
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml:
##########
@@ -2170,6 +2360,386 @@ Sink(table=[default_catalog.default_database.sink], 
fields=[ws, we, b, c])
       +- Exchange(distribution=[hash[b]])
          +- Calc(select=[b, CASE((a = 11), 1, null:INTEGER) AS $f3, PROCTIME() 
AS proctime], where=[SEARCH(a, Sarg[1, 5, 7, 9, 11])])
             +- TableSourceScan(table=[[default_catalog, default_database, 
source, project=[a, b], metadata=[]]], fields=[a, b])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testSession_DistinctSplitEnabled[aggPhaseEnforcer=ONE_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   a,
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   count(distinct c) AS uv
+FROM TABLE(
+  SESSION(TABLE MyTable PARTITION BY a, DESCRIPTOR(proctime), INTERVAL '5' 
MINUTE))
+GROUP BY a, window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], 
EXPR$5=[MAX($3) FILTER $4], uv=[COUNT(DISTINCT $5)])
++- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS 
TRUE(>($1, 1000))], c=[$2])
+   +- LogicalTableFunctionScan(invocation=[SESSION(PARTITION BY($0), 
DESCRIPTOR($6), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT 
b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], 
proctime=[$6])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, 
MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, uv])
++- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[5 
min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, 
MAX(d) FILTER $f4 AS EXPR$5, COUNT(DISTINCT c) AS uv, start('w$) AS 
window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, 
MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, 
rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testSession_DistinctSplitEnabled[aggPhaseEnforcer=TWO_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   a,
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   count(distinct c) AS uv
+FROM TABLE(
+  SESSION(TABLE MyTable PARTITION BY a, DESCRIPTOR(proctime), INTERVAL '5' 
MINUTE))
+GROUP BY a, window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], 
EXPR$5=[MAX($3) FILTER $4], uv=[COUNT(DISTINCT $5)])
++- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS 
TRUE(>($1, 1000))], c=[$2])
+   +- LogicalTableFunctionScan(invocation=[SESSION(PARTITION BY($0), 
DESCRIPTOR($6), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT 
b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], 
proctime=[$6])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, 
MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, uv])
++- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[5 
min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, 
MAX(d) FILTER $f4 AS EXPR$5, COUNT(DISTINCT c) AS uv, start('w$) AS 
window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, 
MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, 
rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSession_OnProctime[aggPhaseEnforcer=TWO_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   a,
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   weightedAvg(b, e) AS wAvg,
+   count(distinct c) AS uv
+FROM TABLE(
+  SESSION(TABLE MyTable PARTITION BY a, DESCRIPTOR(proctime), INTERVAL '5' 
MINUTE))
+GROUP BY a, window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], 
EXPR$5=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)])
++- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS 
TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2])
+   +- LogicalTableFunctionScan(invocation=[SESSION(PARTITION BY($0), 
DESCRIPTOR($6), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT 
b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], 
proctime=[$6])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, 
MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv])
++- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[5 
min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, 
MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS 
uv, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime])
+               +- TableSourceScan(table=[[default_catalog, default_database, 
MyTable]], fields=[a, b, c, d, e, rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSession_OnRowtime[aggPhaseEnforcer=ONE_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   a,
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   weightedAvg(b, e) AS wAvg,
+   count(distinct c) AS uv
+FROM TABLE(
+  SESSION(TABLE MyTable PARTITION BY a, DESCRIPTOR(rowtime), INTERVAL '5' 
MINUTE))
+GROUP BY a, window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], 
EXPR$5=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)])
++- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS 
TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2])
+   +- LogicalTableFunctionScan(invocation=[SESSION(PARTITION BY($0), 
DESCRIPTOR($5), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT 
b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], 
proctime=[$6])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, 
MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv])
++- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[rowtime], gap=[5 
min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, 
MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS 
uv, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+            +- TableSourceScan(table=[[default_catalog, default_database, 
MyTable]], fields=[a, b, c, d, e, rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSession_OnRowtime[aggPhaseEnforcer=TWO_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   a,
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   weightedAvg(b, e) AS wAvg,
+   count(distinct c) AS uv
+FROM TABLE(
+  SESSION(TABLE MyTable PARTITION BY a, DESCRIPTOR(rowtime), INTERVAL '5' 
MINUTE))
+GROUP BY a, window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], 
EXPR$5=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)])
++- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS 
TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2])
+   +- LogicalTableFunctionScan(invocation=[SESSION(PARTITION BY($0), 
DESCRIPTOR($5), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT 
b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], 
proctime=[$6])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, 
MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv])
++- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[rowtime], gap=[5 
min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, 
MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS 
uv, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+            +- TableSourceScan(table=[[default_catalog, default_database, 
MyTable]], fields=[a, b, c, d, e, rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testSessionWindowTVFWithoutPartitionKeyWhenCantMerge[aggPhaseEnforcer=ONE_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   weightedAvg(b, e) AS wAvg,
+   count(distinct c) AS uv
+FROM (
+  SELECT window_start, rowtime, d, proctime, e, b, c, window_end, window_time, 
a
+  FROM TABLE(SESSION(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' MINUTE))
+  WHERE window_start >= TIMESTAMP '2021-01-01 10:10:00.000'
+)
+GROUP BY window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], 
EXPR$4=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)])
++- LogicalProject(window_start=[$0], window_end=[$7], d=[$2], $f3=[IS 
TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6])
+   +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], 
e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0])
+      +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)])
+         +- LogicalTableFunctionScan(invocation=[SESSION(DESCRIPTOR($5), 
300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, 
VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[$6])
+               +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+                     +- LogicalTableScan(table=[[default_catalog, 
default_database, MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv])
++- WindowAggregate(window=[SESSION(win_start=[window_start], 
win_end=[window_end], gap=[5 min])], select=[COUNT(*) AS EXPR$2, SUM(d) AS 
EXPR$3, MAX(d) FILTER $f3 AS EXPR$4, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT 
c) AS uv, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[single])
+      +- Calc(select=[window_start, window_end, d, IS TRUE(>(b, 1000)) AS $f3, 
b, e, c], where=[>=(window_start, 2021-01-01 10:10:00)])
+         +- WindowTableFunction(window=[SESSION(time_col=[rowtime], gap=[5 
min])])
+            +- Exchange(distribution=[single])
+               +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+                  +- TableSourceScan(table=[[default_catalog, 
default_database, MyTable, project=[b, c, d, e, rowtime], metadata=[]]], 
fields=[b, c, d, e, rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testSessionWindowTVFWithoutPartitionKeyWhenCantMerge[aggPhaseEnforcer=TWO_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   window_start,
+   window_end,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   weightedAvg(b, e) AS wAvg,
+   count(distinct c) AS uv
+FROM (
+  SELECT window_start, rowtime, d, proctime, e, b, c, window_end, window_time, 
a
+  FROM TABLE(SESSION(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' MINUTE))
+  WHERE window_start >= TIMESTAMP '2021-01-01 10:10:00.000'
+)
+GROUP BY window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], 
EXPR$4=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)])
++- LogicalProject(window_start=[$0], window_end=[$7], d=[$2], $f3=[IS 
TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6])
+   +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], 
e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0])
+      +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)])
+         +- LogicalTableFunctionScan(invocation=[SESSION(DESCRIPTOR($5), 
300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, 
VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[$6])
+               +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 
1000:INTERVAL SECOND)])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+                     +- LogicalTableScan(table=[[default_catalog, 
default_database, MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv])
++- WindowAggregate(window=[SESSION(win_start=[window_start], 
win_end=[window_end], gap=[5 min])], select=[COUNT(*) AS EXPR$2, SUM(d) AS 
EXPR$3, MAX(d) FILTER $f3 AS EXPR$4, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT 
c) AS uv, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[single])
+      +- Calc(select=[window_start, window_end, d, IS TRUE(>(b, 1000)) AS $f3, 
b, e, c], where=[>=(window_start, 2021-01-01 10:10:00)])
+         +- WindowTableFunction(window=[SESSION(time_col=[rowtime], gap=[5 
min])])
+            +- Exchange(distribution=[single])
+               +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
+                  +- TableSourceScan(table=[[default_catalog, 
default_database, MyTable, project=[b, c, d, e, rowtime], metadata=[]]], 
fields=[b, c, d, e, rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testSessionWindowTVFWithPartitionKeyWhenCantMerge[aggPhaseEnforcer=ONE_PHASE]">
+    <Resource name="sql">
+      <![CDATA[
+SELECT
+   window_start,
+   window_end,
+   a,
+   count(*),
+   sum(d),
+   max(d) filter (where b > 1000),
+   weightedAvg(b, e) AS wAvg,
+   count(distinct c) AS uv
+FROM (
+  SELECT window_start, rowtime, d, proctime, e, b, c, window_end, window_time, 
a
+  FROM TABLE(SESSION(TABLE MyTable PARTITION BY a, DESCRIPTOR(rowtime), 
INTERVAL '5' MINUTE))
+  WHERE window_start >= TIMESTAMP '2021-01-01 10:10:00.000'
+)
+GROUP BY a, window_start, window_end
+      ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(window_start=[$1], window_end=[$2], a=[$0], EXPR$3=[$3], 
EXPR$4=[$4], EXPR$5=[$5], wAvg=[$6], uv=[$7])
++- LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], 
EXPR$5=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)])
+   +- LogicalProject(a=[$9], window_start=[$0], window_end=[$7], d=[$2], 
$f4=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6])
+      +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], 
proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], 
a=[$0])
+         +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)])
+            +- LogicalTableFunctionScan(invocation=[SESSION(PARTITION BY($0), 
DESCRIPTOR($5), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT 
b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* 
rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, 
TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[$6])
+                  +- LogicalWatermarkAssigner(rowtime=[rowtime], 
watermark=[-($5, 1000:INTERVAL SECOND)])
+                     +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], 
rowtime=[$5], proctime=[PROCTIME()])
+                        +- LogicalTableScan(table=[[default_catalog, 
default_database, MyTable]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[window_start, window_end, a, EXPR$3, EXPR$4, EXPR$5, wAvg, uv])
++- WindowAggregate(groupBy=[a], window=[SESSION(win_start=[window_start], 
win_end=[window_end], gap=[5 min], partition keys=[a])], select=[a, COUNT(*) AS 
EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS 
wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS 
window_end])
+   +- Exchange(distribution=[hash[a]])

Review Comment:
   Can we remove this exchange, only retain the first exchange is enough.



##########
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml:
##########
@@ -1186,6 +1186,196 @@ Calc(select=[window_time, start_time, end_time])
          +- Calc(select=[CAST(rowtime AS TIMESTAMP(3)) AS rowtime, rowtime AS 
rowtime_0])
             +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 
1000:INTERVAL SECOND)])
                +- TableSourceScan(table=[[default_catalog, default_database, 
MyTable, project=[rowtime], metadata=[]]], fields=[rowtime])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testGroupKeyLessThanPartitionKeyInSessionWindow[aggPhaseEnforcer=ONE_PHASE]">

Review Comment:
   I think we should add a session window agg test without partition key and 
extra group key to cover the shuffle case of singleton.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);
+        counterState = getRuntimeContext().getState(counterStateDescriptor);
+
+        MapStateDescriptor<Long, RowData> windowStateDescriptor =
+                new MapStateDescriptor<>("window-table-accs", Types.LONG, 
inputRowType);
+
+        windowState =
+                (InternalMapState<RowData, TimeWindow, Long, RowData>)
+                        getOrCreateKeyedState(windowSerializer, 
windowStateDescriptor);
+
+        windowAggregator = new DummyWindowAggregator();
+        windowAggregator.open(
+                new PerWindowStateDataViewStore(
+                        getKeyedStateBackend(), windowSerializer, 
getRuntimeContext()));
+
+        WindowContextImpl windowContext =
+                new WindowContextImpl(
+                        shiftTimeZone,
+                        internalTimerService,
+                        windowAggregator,
+                        windowAssigner,
+                        triggerContext);
+
+        windowFunction =
+                new MergingWindowProcessFunction<>(
+                        (MergingWindowAssigner<TimeWindow>) windowAssigner,
+                        windowAggregator,
+                        windowSerializer,
+                        0,
+                        new MergingConsumer());
+
+        windowFunction.open(windowContext);
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (windowAggregator != null) {

Review Comment:
   call super.close firstly



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/operator/UnalignedWindowTableFunctionOperator.java:
##########
@@ -0,0 +1,496 @@
+/*
+ * 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.runtime.operators.window.tvf.operator;
+
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalMapState;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
+import 
org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.util.TimeWindowUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The operator for unaligned window table function.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ * 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowOperatorBase}.
+ *
+ * <p>Note: The operator only applies for Window TVF with set semantics (e.g 
SESSION) instead of row
+ * semantics (e.g TUMBLE/HOP/CUMULATE).
+ *
+ * <p>This operator emits result at the end of window instead of per record.
+ *
+ * <p>This operator will not compact changelog records.
+ *
+ * <p>This operator will keep the original order of input records when 
outputting.
+ */
+public class UnalignedWindowTableFunctionOperator extends 
WindowTableFunctionOperatorBase
+        implements Triggerable<RowData, TimeWindow> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final Trigger<TimeWindow> trigger;
+
+    private final LogicalType[] inputFieldTypes;
+
+    private final TypeSerializer<TimeWindow> windowSerializer;
+
+    private transient InternalTimerService<TimeWindow> internalTimerService;
+
+    // a counter to tag the order of all input streams when entering the 
operator
+    private transient ValueState<Long> counterState;
+
+    private transient InternalMapState<RowData, TimeWindow, Long, RowData> 
windowState;
+
+    private transient TriggerContextImpl triggerContext;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    protected transient NamespaceAggsHandleFunctionBase<TimeWindow> 
windowAggregator;
+
+    public UnalignedWindowTableFunctionOperator(
+            GroupWindowAssigner<TimeWindow> windowAssigner,
+            TypeSerializer<TimeWindow> windowSerializer,
+            LogicalType[] inputFieldTypes,
+            int rowtimeIndex,
+            ZoneId shiftTimeZone) {
+        super(windowAssigner, rowtimeIndex, shiftTimeZone);
+        this.trigger = createTrigger(windowAssigner);
+        this.windowSerializer = checkNotNull(windowSerializer);
+        this.inputFieldTypes = checkNotNull(inputFieldTypes);
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        internalTimerService =
+                getInternalTimerService("window-table-functions", 
windowSerializer, this);
+
+        triggerContext =
+                new TriggerContextImpl(
+                        trigger, internalTimerService, shiftTimeZone, 
windowSerializer);
+        triggerContext.open();
+
+        TypeInformation<RowData> inputRowType = 
InternalTypeInfo.ofFields(inputFieldTypes);
+
+        ValueStateDescriptor<Long> counterStateDescriptor =
+                new ValueStateDescriptor<>("window-table-counter", Types.LONG, 
0L);
+        counterState = getRuntimeContext().getState(counterStateDescriptor);
+
+        MapStateDescriptor<Long, RowData> windowStateDescriptor =
+                new MapStateDescriptor<>("window-table-accs", Types.LONG, 
inputRowType);
+
+        windowState =
+                (InternalMapState<RowData, TimeWindow, Long, RowData>)
+                        getOrCreateKeyedState(windowSerializer, 
windowStateDescriptor);
+
+        windowAggregator = new DummyWindowAggregator();
+        windowAggregator.open(
+                new PerWindowStateDataViewStore(
+                        getKeyedStateBackend(), windowSerializer, 
getRuntimeContext()));
+
+        WindowContextImpl windowContext =
+                new WindowContextImpl(
+                        shiftTimeZone,
+                        internalTimerService,
+                        windowAggregator,
+                        windowAssigner,
+                        triggerContext);
+
+        windowFunction =
+                new MergingWindowProcessFunction<>(
+                        (MergingWindowAssigner<TimeWindow>) windowAssigner,
+                        windowAggregator,
+                        windowSerializer,
+                        0,
+                        new MergingConsumer());
+
+        windowFunction.open(windowContext);
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (windowAggregator != null) {
+            windowAggregator.close();
+        }
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{

Review Comment:
   How we deal with the late record?If we drop it, I think we should add a 
related metrics.



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/WindowTableFunctionITCase.scala:
##########
@@ -1327,4 +1328,146 @@ class WindowTableFunctionITCase extends BatchTestBase {
       )
     )
   }
+
+  @Test
+  def testSessionTVF(): Unit = {

Review Comment:
   I have one question, do we support session window agg in batch mode?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to