dianfu commented on a change in pull request #13415:
URL: https://github.com/apache/flink/pull/13415#discussion_r490711116



##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.python.aggregate.arrow.batch;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.grouping.HeapWindowsGrouping;
+import 
org.apache.flink.table.runtime.operators.window.grouping.WindowsGrouping;
+import org.apache.flink.table.runtime.util.RowIterator;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.LinkedList;
+
+/**
+ * The Batch Arrow Python {@link AggregateFunction} Operator for Group Window 
Aggregation.
+ */
+@Internal
+public class BatchArrowPythonGroupWindowAggregateFunctionOperator
+       extends AbstractBatchArrowPythonAggregateFunctionOperator {
+
+       private static final long serialVersionUID = 1L;
+
+       /**
+        * The Infos of the Window.
+        * 0 -> start of the Window.
+        * 1 -> end of the Window.
+        * 2 -> row time of the Window.
+        */
+       private final int[] namedProperties;
+
+       /**
+        * The row time index of the input data.
+        */
+       private final int inputTimeFieldIndex;
+
+       /**
+        * The window elements buffer size limit used in group window agg 
operator.
+        */
+       private final int maxLimitSize;
+
+       /**
+        * The window size of the window.
+        */
+       private final long windowSize;
+
+       /**
+        * The sliding size of the sliding window.
+        */
+       private final long slideSize;
+
+       private transient WindowsGrouping windowsGrouping;
+
+       /**
+        * The GenericRowData reused holding the property of the window, such 
as window start, window
+        * end and window time.
+        */
+       private transient GenericRowData windowProperty;
+
+       /**
+        * The JoinedRowData reused holding the window agg execution result.
+        */
+       private transient JoinedRowData windowAggResult;
+
+       /**
+        * The queue holding the input groupSet with the TimeWindow for which 
the execution results
+        * have not been received.
+        */
+       private transient LinkedList<Tuple2<RowData, TimeWindow>> 
inputKeyAndWindow;
+
+       public BatchArrowPythonGroupWindowAggregateFunctionOperator(
+               Configuration config,
+               PythonFunctionInfo[] pandasAggFunctions,
+               RowType inputType,
+               RowType outputType,
+               int inputTimeFieldIndex,
+               int maxLimitSize,
+               long windowSize,
+               long slideSize,
+               int[] namedProperties,
+               int[] groupKey,
+               int[] groupingSet,
+               int[] udafInputOffsets) {
+               super(config, pandasAggFunctions, inputType, outputType, 
groupKey, groupingSet, udafInputOffsets);
+               this.namedProperties = namedProperties;
+               this.inputTimeFieldIndex = inputTimeFieldIndex;
+               this.maxLimitSize = maxLimitSize;
+               this.windowSize = windowSize;
+               this.slideSize = slideSize;
+       }
+
+       @Override
+       public void open() throws Exception {
+               userDefinedFunctionOutputType = new RowType(
+                       outputType.getFields().subList(groupingSet.length, 
outputType.getFieldCount() - namedProperties.length));
+               super.open();

Review comment:
       Putting super.open() at the end of this method? Although it doesn't 
affect the correctness, it makes the code more readable.

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.python.aggregate.arrow.batch;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.grouping.HeapWindowsGrouping;
+import 
org.apache.flink.table.runtime.operators.window.grouping.WindowsGrouping;
+import org.apache.flink.table.runtime.util.RowIterator;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.LinkedList;
+
+/**
+ * The Batch Arrow Python {@link AggregateFunction} Operator for Group Window 
Aggregation.
+ */
+@Internal
+public class BatchArrowPythonGroupWindowAggregateFunctionOperator
+       extends AbstractBatchArrowPythonAggregateFunctionOperator {
+
+       private static final long serialVersionUID = 1L;
+
+       /**
+        * The Infos of the Window.
+        * 0 -> start of the Window.
+        * 1 -> end of the Window.
+        * 2 -> row time of the Window.
+        */
+       private final int[] namedProperties;
+
+       /**
+        * The row time index of the input data.
+        */
+       private final int inputTimeFieldIndex;
+
+       /**
+        * The window elements buffer size limit used in group window agg 
operator.
+        */
+       private final int maxLimitSize;
+
+       /**
+        * The window size of the window.
+        */
+       private final long windowSize;
+
+       /**
+        * The sliding size of the sliding window.
+        */
+       private final long slideSize;
+
+       private transient WindowsGrouping windowsGrouping;
+
+       /**
+        * The GenericRowData reused holding the property of the window, such 
as window start, window
+        * end and window time.
+        */
+       private transient GenericRowData windowProperty;
+
+       /**
+        * The JoinedRowData reused holding the window agg execution result.
+        */
+       private transient JoinedRowData windowAggResult;
+
+       /**
+        * The queue holding the input groupSet with the TimeWindow for which 
the execution results
+        * have not been received.
+        */
+       private transient LinkedList<Tuple2<RowData, TimeWindow>> 
inputKeyAndWindow;
+
+       public BatchArrowPythonGroupWindowAggregateFunctionOperator(
+               Configuration config,
+               PythonFunctionInfo[] pandasAggFunctions,
+               RowType inputType,
+               RowType outputType,
+               int inputTimeFieldIndex,
+               int maxLimitSize,
+               long windowSize,
+               long slideSize,
+               int[] namedProperties,
+               int[] groupKey,
+               int[] groupingSet,
+               int[] udafInputOffsets) {
+               super(config, pandasAggFunctions, inputType, outputType, 
groupKey, groupingSet, udafInputOffsets);
+               this.namedProperties = namedProperties;
+               this.inputTimeFieldIndex = inputTimeFieldIndex;
+               this.maxLimitSize = maxLimitSize;
+               this.windowSize = windowSize;
+               this.slideSize = slideSize;
+       }
+
+       @Override
+       public void open() throws Exception {
+               userDefinedFunctionOutputType = new RowType(
+                       outputType.getFields().subList(groupingSet.length, 
outputType.getFieldCount() - namedProperties.length));
+               super.open();
+               inputKeyAndWindow = new LinkedList<>();
+               windowProperty = new GenericRowData(namedProperties.length);
+               windowAggResult = new JoinedRowData();
+               windowsGrouping = new HeapWindowsGrouping(
+                       maxLimitSize, windowSize, slideSize, 
inputTimeFieldIndex, false);
+       }
+
+       @Override
+       public void close() throws Exception {
+               super.close();
+               windowsGrouping.close();
+       }
+
+       @Override
+       public void bufferInput(RowData input) throws Exception {
+               // always copy the projection result as the generated 
Projection reuses the projection result
+               BinaryRowData currentKey = 
groupKeyProjection.apply(input).copy();
+               currentKey.setRowKind(input.getRowKind());

Review comment:
       Could we refactor the following code a bit to avoid duplication?

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperator.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.python.aggregate.arrow.batch;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.JoinedRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import 
org.apache.flink.table.runtime.operators.window.grouping.HeapWindowsGrouping;
+import 
org.apache.flink.table.runtime.operators.window.grouping.WindowsGrouping;
+import org.apache.flink.table.runtime.util.RowIterator;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.LinkedList;
+
+/**
+ * The Batch Arrow Python {@link AggregateFunction} Operator for Group Window 
Aggregation.
+ */
+@Internal
+public class BatchArrowPythonGroupWindowAggregateFunctionOperator
+       extends AbstractBatchArrowPythonAggregateFunctionOperator {
+
+       private static final long serialVersionUID = 1L;
+
+       /**
+        * The Infos of the Window.
+        * 0 -> start of the Window.
+        * 1 -> end of the Window.
+        * 2 -> row time of the Window.
+        */
+       private final int[] namedProperties;
+
+       /**
+        * The row time index of the input data.
+        */
+       private final int inputTimeFieldIndex;
+
+       /**
+        * The window elements buffer size limit used in group window agg 
operator.
+        */
+       private final int maxLimitSize;
+
+       /**
+        * The window size of the window.
+        */
+       private final long windowSize;
+
+       /**
+        * The sliding size of the sliding window.
+        */
+       private final long slideSize;
+
+       private transient WindowsGrouping windowsGrouping;
+
+       /**
+        * The GenericRowData reused holding the property of the window, such 
as window start, window
+        * end and window time.
+        */
+       private transient GenericRowData windowProperty;
+
+       /**
+        * The JoinedRowData reused holding the window agg execution result.
+        */
+       private transient JoinedRowData windowAggResult;
+
+       /**
+        * The queue holding the input groupSet with the TimeWindow for which 
the execution results
+        * have not been received.
+        */
+       private transient LinkedList<Tuple2<RowData, TimeWindow>> 
inputKeyAndWindow;
+
+       public BatchArrowPythonGroupWindowAggregateFunctionOperator(
+               Configuration config,
+               PythonFunctionInfo[] pandasAggFunctions,
+               RowType inputType,
+               RowType outputType,
+               int inputTimeFieldIndex,
+               int maxLimitSize,
+               long windowSize,
+               long slideSize,
+               int[] namedProperties,
+               int[] groupKey,
+               int[] groupingSet,
+               int[] udafInputOffsets) {
+               super(config, pandasAggFunctions, inputType, outputType, 
groupKey, groupingSet, udafInputOffsets);
+               this.namedProperties = namedProperties;
+               this.inputTimeFieldIndex = inputTimeFieldIndex;
+               this.maxLimitSize = maxLimitSize;
+               this.windowSize = windowSize;
+               this.slideSize = slideSize;
+       }
+
+       @Override
+       public void open() throws Exception {
+               userDefinedFunctionOutputType = new RowType(
+                       outputType.getFields().subList(groupingSet.length, 
outputType.getFieldCount() - namedProperties.length));
+               super.open();
+               inputKeyAndWindow = new LinkedList<>();
+               windowProperty = new GenericRowData(namedProperties.length);
+               windowAggResult = new JoinedRowData();
+               windowsGrouping = new HeapWindowsGrouping(
+                       maxLimitSize, windowSize, slideSize, 
inputTimeFieldIndex, false);
+       }
+
+       @Override
+       public void close() throws Exception {
+               super.close();
+               windowsGrouping.close();
+       }
+
+       @Override
+       public void bufferInput(RowData input) throws Exception {
+               // always copy the projection result as the generated 
Projection reuses the projection result
+               BinaryRowData currentKey = 
groupKeyProjection.apply(input).copy();
+               currentKey.setRowKind(input.getRowKind());
+               if (lastGroupKey == null) {
+                       lastGroupKey = currentKey;
+                       lastGroupSet = groupSetProjection.apply(input).copy();
+               } else if (isNewKey(currentKey)) {
+                       invokeCurrentBatch();
+                       lastGroupKey = currentKey;
+                       lastGroupSet = groupSetProjection.apply(input).copy();
+               }
+       }
+
+       @Override
+       protected void invokeCurrentBatch() throws Exception {
+               windowsGrouping.advanceWatermarkToTriggerAllWindows();
+               triggerWindowProcess();
+               windowsGrouping.reset();
+       }
+
+       @Override
+       public void processElementInternal(RowData value) throws Exception {
+               windowsGrouping.addInputToBuffer((BinaryRowData) value);
+               triggerWindowProcess();
+       }
+
+       @Override
+       @SuppressWarnings("ConstantConditions")
+       public void emitResult(Tuple2<byte[], Integer> resultTuple) throws 
Exception {
+               byte[] udafResult = resultTuple.f0;
+               int length = resultTuple.f1;
+               bais.setBuffer(udafResult, 0, length);
+               int rowCount = arrowSerializer.load();
+               for (int i = 0; i < rowCount; i++) {
+                       Tuple2<RowData, TimeWindow> input = 
inputKeyAndWindow.poll();
+                       RowData key = input.f0;
+                       TimeWindow window = input.f1;
+                       setWindowProperty(window);
+                       windowAggResult.replace(key, arrowSerializer.read(i));
+                       
rowDataWrapper.collect(reuseJoinedRow.replace(windowAggResult, windowProperty));
+               }
+       }
+
+       private void triggerWindowProcess() throws Exception {
+               while (windowsGrouping.hasTriggerWindow()) {
+                       RowIterator<BinaryRowData> elementIterator =
+                               
windowsGrouping.buildTriggerWindowElementsIterator();
+                       TimeWindow currentWindow = 
windowsGrouping.getTriggerWindow();

Review comment:
       Move this line inside `if (currentBatchCount > 0)`?

##########
File path: 
flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.python.aggregate.arrow.batch;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.python.PythonOptions;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import 
org.apache.flink.table.runtime.operators.python.aggregate.arrow.AbstractArrowPythonAggregateFunctionOperator;
+import 
org.apache.flink.table.runtime.operators.python.aggregate.arrow.ArrowPythonAggregateFunctionOperatorTestBase;
+import 
org.apache.flink.table.runtime.utils.PassThroughPythonAggregateFunctionRunner;
+import org.apache.flink.table.runtime.utils.PythonTestUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * Test for {@link BatchArrowPythonGroupWindowAggregateFunctionOperatorTest}. 
These test that:
+ *
+ * <ul>
+ * <li>FinishBundle is called when checkpoint is encountered</li>
+ * <li>Watermarks are buffered and only sent to downstream when finishedBundle 
is triggered</li>
+ * </ul>
+ */
+public class BatchArrowPythonGroupWindowAggregateFunctionOperatorTest extends 
ArrowPythonAggregateFunctionOperatorTestBase {
+       @Test
+       public void testGroupAggregateFunction() throws Exception {
+               OneInputStreamOperatorTestHarness<RowData, RowData> testHarness 
= getTestHarness(
+                       new Configuration());
+               long initialTime = 0L;
+               ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+               testHarness.open();
+
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c2", 0L, 0L), initialTime + 1));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c4", 1L, 6000L), initialTime + 2));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c6", 2L, 10000L), initialTime + 3));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c2", "c8", 3L, 0L), initialTime + 3));
+
+               testHarness.close();
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 1L, 
TimestampData.fromEpochMillis(5000L), TimestampData.fromEpochMillis(15000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 2L, 
TimestampData.fromEpochMillis(10000L), TimestampData.fromEpochMillis(20000L))));
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+       }
+
+       @Test
+       public void testFinishBundleTriggeredOnCheckpoint() throws Exception {
+               Configuration conf = new Configuration();
+               conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 10);
+               OneInputStreamOperatorTestHarness<RowData, RowData> testHarness 
= getTestHarness(conf);
+
+               long initialTime = 0L;
+               ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+               testHarness.open();
+
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c2", 0L, 0L), initialTime + 1));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c4", 1L, 6000L), initialTime + 2));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c6", 2L, 10000L), initialTime + 3));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c2", "c8", 3L, 0L), initialTime + 3));
+               // checkpoint trigger finishBundle
+               testHarness.prepareSnapshotPreBarrier(0L);
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 1L, 
TimestampData.fromEpochMillis(5000L), TimestampData.fromEpochMillis(15000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 2L, 
TimestampData.fromEpochMillis(10000L), TimestampData.fromEpochMillis(20000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+               testHarness.close();
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+       }
+
+       @Test
+       public void testFinishBundleTriggeredByCount() throws Exception {
+               Configuration conf = new Configuration();
+               conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 6);
+               OneInputStreamOperatorTestHarness<RowData, RowData> testHarness 
= getTestHarness(conf);
+
+               long initialTime = 0L;
+               ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+               testHarness.open();
+
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c2", 0L, 0L), initialTime + 1));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c4", 1L, 6000L), initialTime + 2));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c6", 2L, 10000L), initialTime + 3));
+
+               assertOutputEquals("FinishBundle should not be triggered.", 
expectedOutput, testHarness.getOutput());
+
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c2", "c8", 3L, 0L), initialTime + 3));
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 1L, 
TimestampData.fromEpochMillis(5000L), TimestampData.fromEpochMillis(15000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 2L, 
TimestampData.fromEpochMillis(10000L), TimestampData.fromEpochMillis(20000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+               testHarness.close();
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+       }
+
+       @Test
+       public void testFinishBundleTriggeredByTime() throws Exception {
+               Configuration conf = new Configuration();
+               conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 10);
+               conf.setLong(PythonOptions.MAX_BUNDLE_TIME_MILLS, 1000L);
+               OneInputStreamOperatorTestHarness<RowData, RowData> testHarness 
= getTestHarness(conf);
+
+               long initialTime = 0L;
+               ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+               testHarness.open();
+
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c2", 0L, 0L), initialTime + 1));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c4", 1L, 6000L), initialTime + 2));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c6", 2L, 10000L), initialTime + 3));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c2", "c8", 3L, 0L), initialTime + 3));
+               assertOutputEquals("FinishBundle should not be triggered.", 
expectedOutput, testHarness.getOutput());
+
+               testHarness.setProcessingTime(1000L);
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 1L, 
TimestampData.fromEpochMillis(5000L), TimestampData.fromEpochMillis(15000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 2L, 
TimestampData.fromEpochMillis(10000L), TimestampData.fromEpochMillis(20000L))));
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+
+               testHarness.close();
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+       }
+
+       @Test
+       public void testWatermarkProcessedOnFinishBundle() throws Exception {

Review comment:
       BatchArrowPythonGroupWindowAggregateFunctionOperator is running in batch 
and so I guess we don't need to consider watermark?

##########
File path: 
flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.python.aggregate.arrow.batch;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.python.PythonOptions;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import 
org.apache.flink.table.runtime.operators.python.aggregate.arrow.AbstractArrowPythonAggregateFunctionOperator;
+import 
org.apache.flink.table.runtime.operators.python.aggregate.arrow.ArrowPythonAggregateFunctionOperatorTestBase;
+import 
org.apache.flink.table.runtime.utils.PassThroughPythonAggregateFunctionRunner;
+import org.apache.flink.table.runtime.utils.PythonTestUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * Test for {@link BatchArrowPythonGroupWindowAggregateFunctionOperatorTest}. 
These test that:

Review comment:
       ```suggestion
    * Test for {@link BatchArrowPythonGroupWindowAggregateFunctionOperator}. 
These test that:
   ```

##########
File path: 
flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.python.aggregate.arrow.batch;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.python.PythonOptions;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import 
org.apache.flink.table.runtime.operators.python.aggregate.arrow.AbstractArrowPythonAggregateFunctionOperator;
+import 
org.apache.flink.table.runtime.operators.python.aggregate.arrow.ArrowPythonAggregateFunctionOperatorTestBase;
+import 
org.apache.flink.table.runtime.utils.PassThroughPythonAggregateFunctionRunner;
+import org.apache.flink.table.runtime.utils.PythonTestUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * Test for {@link BatchArrowPythonGroupWindowAggregateFunctionOperatorTest}. 
These test that:
+ *
+ * <ul>
+ * <li>FinishBundle is called when checkpoint is encountered</li>
+ * <li>Watermarks are buffered and only sent to downstream when finishedBundle 
is triggered</li>
+ * </ul>
+ */
+public class BatchArrowPythonGroupWindowAggregateFunctionOperatorTest extends 
ArrowPythonAggregateFunctionOperatorTestBase {
+       @Test
+       public void testGroupAggregateFunction() throws Exception {
+               OneInputStreamOperatorTestHarness<RowData, RowData> testHarness 
= getTestHarness(
+                       new Configuration());
+               long initialTime = 0L;
+               ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+               testHarness.open();
+
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c2", 0L, 0L), initialTime + 1));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c4", 1L, 6000L), initialTime + 2));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c1", "c6", 2L, 10000L), initialTime + 3));
+               testHarness.processElement(new 
StreamRecord<>(newBinaryRow(true, "c2", "c8", 3L, 0L), initialTime + 3));
+
+               testHarness.close();
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 0L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 1L, 
TimestampData.fromEpochMillis(5000L), TimestampData.fromEpochMillis(15000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c1", 2L, 
TimestampData.fromEpochMillis(10000L), TimestampData.fromEpochMillis(20000L))));
+
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(-5000L), TimestampData.fromEpochMillis(5000L))));
+               expectedOutput.add(new StreamRecord<>(
+                       newRow(true, "c2", 3L, 
TimestampData.fromEpochMillis(0L), TimestampData.fromEpochMillis(10000L))));
+
+               assertOutputEquals("Output was not correct.", expectedOutput, 
testHarness.getOutput());
+       }
+
+       @Test
+       public void testFinishBundleTriggeredOnCheckpoint() throws Exception {

Review comment:
       BatchArrowPythonGroupWindowAggregateFunctionOperator is running in 
batch, so I guess there is no checkpoint?




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

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


Reply via email to