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


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnsliceAssigners.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.unslicing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.SessionWindowAssigner;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.tvf.common.ClockService;
+
+import java.time.Duration;
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Optional;
+
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Utilities to create {@link UnsliceAssigner}s. */
+@Internal
+public class UnsliceAssigners {
+
+    /**
+     * Creates a session window {@link UnsliceAssigner} that assigns elements 
to windows based on
+     * the timestamp.
+     *
+     * @param rowtimeIndex The index of rowtime field in the input row, {@code 
-1} if based on
+     *     processing time.
+     * @param shiftTimeZone The shift timezone of the window, if the proctime 
or rowtime type is
+     *     TIMESTAMP_LTZ, the shift timezone is the timezone user configured 
in TableConfig, other
+     *     cases the timezone is UTC which means never shift when assigning 
windows.
+     * @param gap The session timeout, i.e. the time gap between sessions
+     */
+    public static SessionUnsliceAssigner session(
+            int rowtimeIndex, ZoneId shiftTimeZone, Duration gap) {
+        return new SessionUnsliceAssigner(rowtimeIndex, shiftTimeZone, 
gap.toMillis());
+    }
+
+    /** The {@link UnsliceAssigner} for session windows. */
+    public static class SessionUnsliceAssigner implements 
UnsliceAssigner<TimeWindow> {
+
+        private static final long serialVersionUID = 1L;
+
+        private final int rowtimeIndex;
+        private final boolean isEventTime;
+        private final ZoneId shiftTimeZone;
+
+        private final SessionWindowAssigner innerSessionWindowAssigner;
+
+        public SessionUnsliceAssigner(int rowtimeIndex, ZoneId shiftTimeZone, 
long sessionGap) {
+            this.rowtimeIndex = rowtimeIndex;
+            this.shiftTimeZone = shiftTimeZone;
+            this.isEventTime = rowtimeIndex >= 0;
+            this.innerSessionWindowAssigner =
+                    
SessionWindowAssigner.withGap(Duration.ofMillis(sessionGap));
+            if (isEventTime()) {
+                this.innerSessionWindowAssigner.withEventTime();
+            } else {
+                this.innerSessionWindowAssigner.withProcessingTime();
+            }
+        }
+
+        @Override
+        public MergingWindowAssigner<TimeWindow> getMergingWindowAssigner() {
+            return innerSessionWindowAssigner;
+        }
+
+        @Override
+        public Optional<TimeWindow> assignActualWindow(
+                RowData element,
+                ClockService clock,
+                MergingWindowProcessFunction<?, TimeWindow> windowFunction)
+                throws Exception {
+            Collection<TimeWindow> windows =
+                    windowFunction.assignActualWindows(element, 
getUtcTimestamp(element, clock));
+            checkState(windows.size() <= 1);
+            if (windows.size() == 1) {
+                return Optional.of(windows.iterator().next());
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        @Override
+        public Optional<TimeWindow> assignStateNamespace(
+                RowData element,
+                ClockService clock,
+                MergingWindowProcessFunction<?, TimeWindow> windowFunction)
+                throws Exception {
+            Collection<TimeWindow> windows =
+                    windowFunction.assignStateNamespace(element, 
getUtcTimestamp(element, clock));
+            checkState(windows.size() <= 1);
+            if (windows.size() == 1) {
+                return Optional.of(windows.iterator().next());
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        protected long getUtcTimestamp(RowData element, ClockService clock) {
+            final long timestamp;
+            if (rowtimeIndex >= 0) {
+                if (element.isNullAt(rowtimeIndex)) {
+                    throw new RuntimeException(
+                            "RowTime field should not be null,"

Review Comment:
   It would be better also throw the `rowtimeIndex`.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceWindowAggProcessor.java:
##########
@@ -0,0 +1,391 @@
+/*
+ * 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.aggregate.window.processors;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.MergingState;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalMergingState;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.RowDataUtil;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+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.common.WindowTimerService;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigner;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowProcessor;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowTimerServiceImpl;
+import org.apache.flink.util.Preconditions;
+
+import java.time.ZoneId;
+import java.util.Collection;
+import java.util.Optional;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+
+/**
+ * An window aggregate processor implementation which works for {@link 
UnsliceAssigner}, e.g.
+ * session windows.
+ */
+public class UnsliceWindowAggProcessor extends 
AbstractWindowAggProcessor<TimeWindow>
+        implements UnslicingWindowProcessor<TimeWindow> {
+
+    private final UnsliceAssigner<TimeWindow> unsliceAssigner;
+
+    private final Trigger<TimeWindow> trigger;
+
+    // 
----------------------------------------------------------------------------------------
+
+    private transient MetricGroup metrics;
+
+    protected transient MergingWindowProcessFunction<RowData, TimeWindow> 
windowFunction;
+
+    private transient TriggerContextImpl triggerContext;
+
+    public UnsliceWindowAggProcessor(
+            GeneratedNamespaceAggsHandleFunction<TimeWindow> genAggsHandler,
+            UnsliceAssigner<TimeWindow> unsliceAssigner,
+            TypeSerializer<RowData> accSerializer,
+            int indexOfCountStar,
+            ZoneId shiftTimeZone) {
+        super(
+                genAggsHandler,
+                unsliceAssigner,
+                accSerializer,
+                unsliceAssigner.isEventTime(),
+                indexOfCountStar,
+                shiftTimeZone);
+        this.unsliceAssigner = unsliceAssigner;
+        if (isEventTime) {
+            trigger = EventTimeTriggers.afterEndOfWindow();
+        } else {
+            trigger = ProcessingTimeTriggers.afterEndOfWindow();
+        }
+    }
+
+    @Override
+    public void open(Context<TimeWindow> context) throws Exception {
+        super.open(context);
+        this.metrics = context.getRuntimeContext().getMetricGroup();
+        this.windowFunction =
+                new MergingWindowProcessFunction<>(
+                        unsliceAssigner.getMergingWindowAssigner(),
+                        aggregator,
+                        unsliceAssigner
+                                .getMergingWindowAssigner()
+                                .getWindowSerializer(new ExecutionConfig()),
+                        // TODO support allowedLateness
+                        0L);
+
+        triggerContext = new TriggerContextImpl();
+        triggerContext.open();
+
+        WindowContextImpl windowContext = new WindowContextImpl();
+
+        this.windowFunction.open(windowContext);
+    }
+
+    @Override
+    public boolean processElement(RowData key, RowData element) throws 
Exception {
+        // the windows which the input row should be placed into
+        Optional<TimeWindow> affectedWindowOp =
+                unsliceAssigner.assignStateNamespace(element, clockService, 
windowFunction);
+        boolean isElementDropped = true;
+        if (affectedWindowOp.isPresent()) {
+            TimeWindow affectedWindow = affectedWindowOp.get();
+            isElementDropped = false;
+
+            RowData acc = windowState.value(affectedWindow);
+            if (acc == null) {
+                acc = aggregator.createAccumulators();
+            }
+            aggregator.setAccumulators(affectedWindow, acc);
+
+            if (RowDataUtil.isAccumulateMsg(element)) {
+                aggregator.accumulate(element);
+            } else {
+                aggregator.retract(element);
+            }
+            acc = aggregator.getAccumulators();
+            windowState.update(affectedWindow, acc);
+        }
+
+        // the actual window which the input row is belongs to
+        Optional<TimeWindow> actualWindowOp =
+                unsliceAssigner.assignActualWindow(element, clockService, 
windowFunction);
+        Preconditions.checkArgument(
+                (affectedWindowOp.isPresent() && actualWindowOp.isPresent())
+                        || (!affectedWindowOp.isPresent() && 
!actualWindowOp.isPresent()));
+
+        if (actualWindowOp.isPresent()) {
+            TimeWindow actualWindow = actualWindowOp.get();
+            triggerContext.setWindow(actualWindow);
+            // register a timer for the window to fire and clean up
+            long triggerTime = 
toEpochMillsForTimer(actualWindow.maxTimestamp(), shiftTimeZone);
+            if (isEventTime) {
+                triggerContext.registerEventTimeTimer(triggerTime);
+            } else {
+                triggerContext.registerProcessingTimeTimer(triggerTime);
+            }
+        }
+        return isElementDropped;
+    }
+
+    @Override
+    public void fireWindow(long timerTimestamp, TimeWindow window) throws 
Exception {
+        windowFunction.prepareAggregateAccumulatorForEmit(window);
+        RowData aggResult = aggregator.getValue(window);
+        triggerContext.setWindow(window);
+        final boolean isFired;
+        if (isEventTime) {
+            isFired = triggerContext.onEventTime(timerTimestamp);
+        } else {
+            isFired = triggerContext.onProcessingTime(timerTimestamp);
+        }
+        // we shouldn't emit an empty window
+        if (isFired && !emptySupplier.get()) {
+            collect(aggResult);
+        }
+    }
+
+    @Override
+    public void clearWindow(long timerTimestamp, TimeWindow window) throws 
Exception {
+        windowFunction.cleanWindowIfNeeded(window, timerTimestamp);
+    }
+
+    @Override
+    public void advanceProgress(long progress) throws Exception {}
+
+    @Override
+    public void prepareCheckpoint() throws Exception {}
+
+    @Override
+    public TypeSerializer<TimeWindow> createWindowSerializer() {
+        return unsliceAssigner
+                .getMergingWindowAssigner()
+                .getWindowSerializer(new ExecutionConfig());
+    }
+
+    @Override
+    protected WindowTimerService<TimeWindow> getWindowTimerService() {
+        return new UnslicingWindowTimerServiceImpl(ctx.getTimerService(), 
shiftTimeZone);
+    }
+
+    private class WindowContextImpl
+            implements InternalWindowProcessFunction.Context<RowData, 
TimeWindow> {
+
+        @Override
+        public long currentProcessingTime() {
+            return ctx.getTimerService().currentProcessingTime();
+        }
+
+        @Override
+        public long currentWatermark() {
+            return ctx.getTimerService().currentWatermark();
+        }
+
+        @Override
+        public ZoneId getShiftTimeZone() {
+            return shiftTimeZone;
+        }
+
+        @Override
+        public RowData getWindowAccumulators(TimeWindow window) throws 
Exception {
+            return windowState.value(window);
+        }
+
+        @Override
+        public void setWindowAccumulators(TimeWindow window, RowData acc) 
throws Exception {
+            windowState.update(window, acc);
+        }
+
+        @Override
+        public void clearWindowState(TimeWindow window) throws Exception {
+            windowState.clear(window);
+            aggregator.cleanup(window);
+        }
+
+        @Override
+        public void clearPreviousState(TimeWindow window) throws Exception {}
+
+        @Override
+        public void clearTrigger(TimeWindow window) throws Exception {
+            triggerContext.setWindow(window);
+            triggerContext.clear();
+        }
+
+        @Override
+        public void deleteCleanupTimer(TimeWindow window) throws Exception {
+            long cleanupTime = toEpochMillsForTimer(window.maxTimestamp(), 
shiftTimeZone);
+            if (cleanupTime == Long.MAX_VALUE) {
+                // no need to clean up because we didn't set one
+                return;
+            }
+            if (unsliceAssigner.isEventTime()) {
+                triggerContext.deleteEventTimeTimer(cleanupTime);
+            } else {
+                triggerContext.deleteProcessingTimeTimer(cleanupTime);
+            }
+        }
+
+        @Override
+        public void onMerge(TimeWindow newWindow, Collection<TimeWindow> 
mergedWindows)
+                throws Exception {
+            triggerContext.setWindow(newWindow);
+            triggerContext.setMergedWindows(mergedWindows);
+            triggerContext.onMerge();
+        }
+
+        @Override
+        public <S extends State> S getPartitionedState(StateDescriptor<S, ?> 
stateDescriptor)
+                throws Exception {
+            requireNonNull(stateDescriptor, "The state properties must not be 
null");
+            return ctx.getKeyedStateBackend()
+                    .getPartitionedState(
+                            VoidNamespace.INSTANCE,

Review Comment:
   Why the namespace is void?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java:
##########
@@ -74,6 +74,9 @@ public abstract class AbstractWindowAggProcessor<W> 
implements WindowProcessor<W
     /** state schema: [key, window, accumulator]. */
     protected transient WindowValueState<W> windowState;
 
+    /** The inner state in {@link #windowState}. */
+    protected transient InternalValueState<RowData, W, RowData> 
internalWindowState;

Review Comment:
   No one use this member variable, I think we can revert this change.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java:
##########
@@ -169,7 +171,11 @@ protected Transformation<RowData> translateToPlanInternal(
                 TimeWindowUtil.getShiftTimeZone(
                         windowing.getTimeAttributeType(),
                         TableConfigUtils.getLocalTimeZone(config));
-        final SliceAssigner sliceAssigner = createSliceAssigner(windowing, 
shiftTimeZone);
+        final WindowAssigner windowAssigner = createWindowAssigner(windowing, 
shiftTimeZone);

Review Comment:
   What is the behavior if enable the two-phase agg forcibly with session 
window? Assuming the following pattern:
   Local Agg -> Global Agg -> Session Window? Does it can work? 



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/UnslicingWindowAggOperatorTest.java:
##########
@@ -0,0 +1,628 @@
+/*
+ * 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.aggregate.window;
+
+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.EmptyRowDataKeySelector;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigner;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigners;
+import 
org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowOperator;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.runtime.util.GenericRowRecordSortComparator;
+import org.apache.flink.table.runtime.util.RowDataHarnessAssertor;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.types.RowKind;
+
+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;
+
+import static org.apache.flink.table.data.TimestampData.fromEpochMillis;
+import static 
org.apache.flink.table.runtime.util.StreamRecordUtils.binaryRecord;
+import static 
org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for unslicing window aggregate operators created by {@link 
WindowAggOperatorBuilder}. */
+@RunWith(Parameterized.class)
+public class UnslicingWindowAggOperatorTest extends WindowAggOperatorTestBase {
+
+    public UnslicingWindowAggOperatorTest(ZoneId shiftTimeZone) {
+        super(shiftTimeZone);
+    }
+
+    @Test
+    public void testEventTimeSessionWindows() throws Exception {
+        final UnsliceAssigner<TimeWindow> assigner =
+                UnsliceAssigners.session(2, shiftTimeZone, 
Duration.ofSeconds(3));
+
+        final UnslicingSumAndCountAggsFunction aggsFunction =
+                new UnslicingSumAndCountAggsFunction();
+        UnslicingWindowOperator<RowData, ?> operator =
+                (UnslicingWindowOperator<RowData, ?>)
+                        WindowAggOperatorBuilder.builder()
+                                .inputSerializer(INPUT_ROW_SER)
+                                .shiftTimeZone(shiftTimeZone)
+                                .keySerializer(KEY_SER)
+                                .assigner(assigner)
+                                
.aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER)
+                                .build();
+
+        OneInputStreamOperatorTestHarness<RowData, RowData> testHarness =
+                createTestHarness(operator);
+
+        testHarness.setup(OUT_SERIALIZER);
+        testHarness.open();
+
+        // process elements
+        ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+        // add elements out-of-order
+        testHarness.processElement(insertRecord("key2", 1, 
fromEpochMillis(3999L)));
+        testHarness.processElement(insertRecord("key2", 1, 
fromEpochMillis(3000L)));
+
+        testHarness.processElement(insertRecord("key1", 1, 
fromEpochMillis(20L)));
+        testHarness.processElement(insertRecord("key1", 1, 
fromEpochMillis(0L)));
+        testHarness.processElement(insertRecord("key1", 1, 
fromEpochMillis(999L)));
+
+        testHarness.processElement(insertRecord("key2", 1, 
fromEpochMillis(1998L)));
+        testHarness.processElement(insertRecord("key2", 1, 
fromEpochMillis(1999L)));
+        testHarness.processElement(insertRecord("key2", 1, 
fromEpochMillis(1000L)));
+
+        testHarness.processWatermark(new Watermark(999));
+        expectedOutput.add(new Watermark(999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        testHarness.processWatermark(new Watermark(1999));
+        expectedOutput.add(new Watermark(1999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        // do a snapshot, close and restore again
+        testHarness.prepareSnapshotPreBarrier(0L);
+        OperatorSubtaskState snapshot = testHarness.snapshot(0L, 0);
+        testHarness.close();
+
+        assertThat(aggsFunction.closeCalled.get()).as("Close was not 
called.").isGreaterThan(0);
+
+        expectedOutput.clear();
+        testHarness = createTestHarness(operator);
+        testHarness.setup();
+        testHarness.initializeState(snapshot);
+        testHarness.open();
+
+        testHarness.processWatermark(new Watermark(2999));
+        expectedOutput.add(new Watermark(2999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        testHarness.processWatermark(new Watermark(3999));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(0L), 
localMills(3999L)));
+        expectedOutput.add(new Watermark(3999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        // late element
+        testHarness.processElement(insertRecord("key2", 1, 
fromEpochMillis(3500L)));
+
+        testHarness.processWatermark(new Watermark(4999));
+        expectedOutput.add(new Watermark(4999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        // late for all assigned windows, should be dropped
+        testHarness.processElement(insertRecord("key1", 1, 
fromEpochMillis(999L)));
+
+        testHarness.processWatermark(new Watermark(5999));
+        expectedOutput.add(new Watermark(5999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        testHarness.processWatermark(new Watermark(6999));
+        expectedOutput.add(insertRecord("key2", 6L, 6L, localMills(1000L), 
localMills(6999L)));
+        expectedOutput.add(new Watermark(6999));
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        // those don't have any effect...
+        testHarness.processWatermark(new Watermark(7999));
+        testHarness.processWatermark(new Watermark(8999));
+        expectedOutput.add(new Watermark(7999));
+        expectedOutput.add(new Watermark(8999));
+
+        ASSERTER.assertOutputEqualsSorted(
+                "Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+        
assertThat(operator.getNumLateRecordsDropped().getCount()).isEqualTo(1);
+
+        testHarness.close();
+    }
+
+    @Test
+    public void testEventTimeSessionWindowsWithCDC() throws Exception {

Review Comment:
   CDC -> Changelog?



-- 
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