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


##########
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:
   I thinks we should avoid use TypeInformation is table mode, try use 
LogicalType first.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to