danny0405 commented on a change in pull request #13650:
URL: https://github.com/apache/flink/pull/13650#discussion_r507540343



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CumulativeWindowAssigner.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.assigners;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import org.apache.flink.util.IterableIterator;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * A {@link WindowAssigner} that windows elements into cumulative windows 
based on the timestamp of the
+ * elements. Windows are overlap.
+ */
+public class CumulativeWindowAssigner extends PanedWindowAssigner<TimeWindow> 
implements InternalTimeWindowAssigner {
+
+       private static final long serialVersionUID = 4895551155814656518L;
+
+       private final long size;
+
+       private final long step;
+
+       private final long offset;
+
+       private final long paneSize;
+
+       private final boolean isEventTime;
+
+       protected CumulativeWindowAssigner(long size, long step, long offset, 
boolean isEventTime) {
+               if (size <= 0 || step <= 0) {
+                       throw new IllegalArgumentException(
+                               "CumulativeWindowAssigner parameters must 
satisfy step > 0 and size > 0");
+               }
+               if (size % step != 0) {
+                       throw new IllegalArgumentException(
+                               "CumulativeWindowAssigner requires size must be 
an integral multiple of step.");
+               }
+
+               this.size = size;
+               this.step = step;

Review comment:
       Do we support `size` argument that is not `1 day` ? Say `2 days` or `3 
days` ?

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CumulativeWindowAssigner.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.assigners;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import org.apache.flink.util.IterableIterator;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * A {@link WindowAssigner} that windows elements into cumulative windows 
based on the timestamp of the
+ * elements. Windows are overlap.
+ */
+public class CumulativeWindowAssigner extends PanedWindowAssigner<TimeWindow> 
implements InternalTimeWindowAssigner {
+
+       private static final long serialVersionUID = 4895551155814656518L;
+
+       private final long size;
+
+       private final long step;
+
+       private final long offset;
+
+       private final long paneSize;
+
+       private final boolean isEventTime;
+
+       protected CumulativeWindowAssigner(long size, long step, long offset, 
boolean isEventTime) {
+               if (size <= 0 || step <= 0) {
+                       throw new IllegalArgumentException(
+                               "CumulativeWindowAssigner parameters must 
satisfy step > 0 and size > 0");
+               }
+               if (size % step != 0) {
+                       throw new IllegalArgumentException(
+                               "CumulativeWindowAssigner requires size must be 
an integral multiple of step.");
+               }
+
+               this.size = size;
+               this.step = step;
+               this.offset = offset;
+               this.isEventTime = isEventTime;
+               this.paneSize = step;
+       }

Review comment:
       If `paneSize ` always equals `step`, keep only `step` is enough, because 
we always have limitation `size must be an integral multiple of step`.

##########
File path: 
flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorTest.java
##########
@@ -289,6 +289,155 @@ public void testProcessingTimeSlidingWindows() throws 
Throwable {
                testHarness.close();
        }
 
+       @Test
+       public void testEventTimeCumulativeWindows() throws Exception {
+               closeCalled.set(0);
+
+               WindowOperator operator = WindowOperatorBuilder
+                       .builder()
+                       .withInputFields(inputFieldTypes)
+                       .cumulative(Duration.ofSeconds(3), 
Duration.ofSeconds(1))
+                       .withEventTime(2)
+                       .aggregateAndBuild(getTimeWindowAggFunction(), 
equaliser, accTypes, aggResultTypes, windowTypes);
+
+               OneInputStreamOperatorTestHarness<RowData, RowData> testHarness 
= createTestHarness(operator);
+
+               testHarness.open();
+
+               // process elements
+               ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+               // add elements out-of-order
+               testHarness.processElement(insertRecord("key2", 1, 2999L));
+               testHarness.processElement(insertRecord("key2", 1, 3000L));
+
+               testHarness.processElement(insertRecord("key1", 1, 20L));
+               testHarness.processElement(insertRecord("key1", 1, 0L));
+               testHarness.processElement(insertRecord("key1", 1, 999L));
+
+               testHarness.processElement(insertRecord("key2", 1, 1998L));
+               testHarness.processElement(insertRecord("key2", 1, 1999L));
+               testHarness.processElement(insertRecord("key2", 1, 1000L));
+
+               testHarness.processWatermark(new Watermark(999));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key1", 3L, 3L, 0L, 1000L, 999L)));
+               expectedOutput.add(new Watermark(999));
+               assertor.assertOutputEqualsSorted("Output was not correct.", 
expectedOutput, testHarness.getOutput());
+
+               testHarness.processWatermark(new Watermark(1999));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key1", 3L, 3L, 0L, 2000L, 1999L)));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key2", 3L, 3L, 0L, 2000L, 1999L)));
+               expectedOutput.add(new Watermark(1999));
+               assertor.assertOutputEqualsSorted("Output was not correct.", 
expectedOutput, testHarness.getOutput());
+
+               testHarness.processWatermark(new Watermark(2999));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key1", 3L, 3L, 0L, 3000L, 2999L)));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key2", 4L, 4L, 0L, 3000L, 2999L)));
+               expectedOutput.add(new Watermark(2999));
+               assertor.assertOutputEqualsSorted("Output was not correct.", 
expectedOutput, testHarness.getOutput());
+
+               // do a snapshot, close and restore again
+               OperatorSubtaskState snapshot = testHarness.snapshot(0L, 0);
+               testHarness.close();
+               expectedOutput.clear();
+
+               testHarness = createTestHarness(operator);
+               testHarness.setup();
+               testHarness.initializeState(snapshot);
+               testHarness.open();
+
+               testHarness.processWatermark(new Watermark(3999));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key2", 1L, 1L, 3000L, 4000L, 3999L)));
+               expectedOutput.add(new Watermark(3999));
+               assertor.assertOutputEqualsSorted("Output was not correct.", 
expectedOutput, testHarness.getOutput());
+
+               testHarness.processWatermark(new Watermark(4999));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key2", 1L, 1L, 3000L, 5000L, 4999L)));
+               expectedOutput.add(new Watermark(4999));
+               assertor.assertOutputEqualsSorted("Output was not correct.", 
expectedOutput, testHarness.getOutput());
+
+               testHarness.processWatermark(new Watermark(5999));
+               expectedOutput.addAll(doubleRecord(isTableAggregate, 
insertRecord("key2", 1L, 1L, 3000L, 6000L, 5999L)));

Review comment:
       We may also need to add a test case for the `late arrive` data.

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CumulativeWindowAssigner.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.assigners;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
+import org.apache.flink.util.IterableIterator;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * A {@link WindowAssigner} that windows elements into cumulative windows 
based on the timestamp of the
+ * elements. Windows are overlap.
+ */
+public class CumulativeWindowAssigner extends PanedWindowAssigner<TimeWindow> 
implements InternalTimeWindowAssigner {
+
+       private static final long serialVersionUID = 4895551155814656518L;
+
+       private final long size;
+
+       private final long step;
+
+       private final long offset;
+
+       private final long paneSize;
+
+       private final boolean isEventTime;
+
+       protected CumulativeWindowAssigner(long size, long step, long offset, 
boolean isEventTime) {
+               if (size <= 0 || step <= 0) {
+                       throw new IllegalArgumentException(
+                               "CumulativeWindowAssigner parameters must 
satisfy step > 0 and size > 0");
+               }
+               if (size % step != 0) {
+                       throw new IllegalArgumentException(
+                               "CumulativeWindowAssigner requires size must be 
an integral multiple of step.");
+               }
+
+               this.size = size;
+               this.step = step;

Review comment:
       We should name it the `maxSize` because the window size is actually 
increasing.




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