fsk119 commented on code in PR #24068:
URL: https://github.com/apache/flink/pull/24068#discussion_r1455848131
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/WindowSpec.java:
##########
@@ -31,4 +32,15 @@
public interface WindowSpec {
String toSummaryString(String windowing);
+
+ /**
+ * Return if the window is a aligned window.
Review Comment:
Return true if xxx
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link
InternalWindowProcessFunction}. */
+public interface WindowContext<K, W extends Window> {
+
+ /**
+ * Creates a partitioned state handle, using the state backend configured
for this task.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was
already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the
key/value state.
+ */
+ <S extends State> S getPartitionedState(StateDescriptor<S, ?>
stateDescriptor) throws Exception;
+
+ /** @return current key of current processed element. */
+ K currentKey();
+
+ /** Returns the current processing time. */
+ long currentProcessingTime();
+
+ /** Returns the current event-time watermark. */
+ long currentWatermark();
+
+ /** Returns the shifted timezone of the window. */
+ ZoneId getShiftTimeZone();
+
+ /** Gets the accumulators of the given window. */
+ RowData getWindowAccumulators(W window) throws Exception;
+
+ /** Sets the accumulators of the given window. */
+ void setWindowAccumulators(W window, RowData acc) throws Exception;
+
+ /** Clear window state of the given window. */
+ void clearWindowState(W window) throws Exception;
+
+ /** Clear previous agg state (used for retraction) of the given window. */
+ void clearPreviousState(W window) throws Exception;
+
+ /** Call {@link Trigger#clear(Window)}} on trigger. */
+ void clearTrigger(W window) throws Exception;
Review Comment:
BTW, if a context provide some actions, it's better to rename it...
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/windowtvf/common/AbstractWindowOperator.java:
##########
@@ -73,34 +68,18 @@
* <p>Because aligned windows can be divided into finite number of
non-overlapping chunks (a.k.a.
* slices), which can apply efficient processing to share intermediate results.
*
- * <h3>Concept of Slice</h3>
- *
- * <p>Dividing a window of aligned windows into a finite number of
non-overlapping chunks, where the
- * chunks are slices. It has the following properties:
- *
- * <ul>
- * <li>An element must only belong to a single slice.
- * <li>Slices are non-overlapping, i.e. S_i and S_j should not have any
shared elements if i != j.
- * <li>A window is consist of a finite number of slices.
- * </ul>
- *
- * <h3>Abstraction of Slicing Window Operator</h3>
- *
- * <p>A slicing window operator is a simple wrap of {@link
SlicingWindowProcessor}. It delegates all
- * the important methods to the underlying processor, where the processor can
have different
- * implementation for aggregate and topk or others.
+ * <p>Note: currently, {@link AbstractWindowOperator} doesn't support
early-fire and late-arrival.
+ * Thus late elements (elements belong to emitted windows) will be simply
dropped.
*
- * <p>A {@link SlicingWindowProcessor} usually leverages the {@link
SliceAssigner} to assign slices
- * and calculate based on the slices. See {@link
SliceSharedWindowAggProcessor} as an example.
+ * <p>See more details in {@link
+ *
org.apache.flink.table.runtime.operators.window.windowtvf.slicing.SlicingWindowOperator}.
*
- * <p>Note: since {@link SlicingWindowProcessor} leverages slicing
optimization for aligned windows,
- * therefore, it doesn't support unaligned windows, e.g. session window.
+ * <p>TODO support unslicing window operator.
*
- * <p>Note: currently, {@link SlicingWindowOperator} doesn't support
early-fire and late-arrival.
- * Thus late elements (elements belong to emitted windows) will be simply
dropped.
+ * <p>TODO support early / late fire.
Review Comment:
ditto
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/builder/AbstractWindowAggOperatorBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.builder;
+
+import org.apache.flink.table.data.RowData;
+import
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+
+import java.time.ZoneId;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link AbstractWindowAggOperatorBuilder} is a base class for building
window aggregate
+ * operators.
+ *
+ * <p>See more details in {@link SlicingWindowAggOperatorBuilder}.
+ *
+ * <p>TODO support UnslicingWindowAggOperatorBuilder.
+ *
+ * @param <W> The type of the window. {@link Long} for slicing window.
+ * @param <T> The implementation of the abstract builder.
+ */
+public abstract class AbstractWindowAggOperatorBuilder<
Review Comment:
Make it as an inner class?
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/WindowSpec.java:
##########
@@ -31,4 +32,15 @@
public interface WindowSpec {
String toSummaryString(String windowing);
+
+ /**
+ * Return if the window is a aligned window.
+ *
+ * <p>See more details about aligned window and unaligned window in {@link
+ *
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator}.
+ *
+ * <p>TODO introduce unaligned window like session window.
Review Comment:
Remove todo or replace with a jira issue.
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/builder/AbstractWindowAggOperatorBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.builder;
+
+import org.apache.flink.table.data.RowData;
+import
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+
+import java.time.ZoneId;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link AbstractWindowAggOperatorBuilder} is a base class for building
window aggregate
+ * operators.
+ *
+ * <p>See more details in {@link SlicingWindowAggOperatorBuilder}.
+ *
+ * <p>TODO support UnslicingWindowAggOperatorBuilder.
+ *
+ * @param <W> The type of the window. {@link Long} for slicing window.
+ * @param <T> The implementation of the abstract builder.
+ */
+public abstract class AbstractWindowAggOperatorBuilder<
+ W, T extends AbstractWindowAggOperatorBuilder> {
+
+ protected AbstractRowDataSerializer<RowData> inputSerializer;
+ protected PagedTypeSerializer<RowData> keySerializer;
+ protected AbstractRowDataSerializer<RowData> accSerializer;
+ protected GeneratedNamespaceAggsHandleFunction<W>
generatedAggregateFunction;
+ protected GeneratedNamespaceAggsHandleFunction<W>
localGeneratedAggregateFunction;
+ protected GeneratedNamespaceAggsHandleFunction<W>
globalGeneratedAggregateFunction;
+ protected ZoneId shiftTimeZone;
+
+ public T inputSerializer(AbstractRowDataSerializer<RowData>
inputSerializer) {
+ this.inputSerializer = inputSerializer;
+ return self();
+ }
+
+ public T shiftTimeZone(ZoneId shiftTimeZone) {
+ this.shiftTimeZone = shiftTimeZone;
+ return self();
+ }
+
+ public T keySerializer(PagedTypeSerializer<RowData> keySerializer) {
+ this.keySerializer = keySerializer;
+ return self();
+ }
+
+ public T aggregate(
+ GeneratedNamespaceAggsHandleFunction<W> generatedAggregateFunction,
+ AbstractRowDataSerializer<RowData> accSerializer) {
+ this.generatedAggregateFunction = generatedAggregateFunction;
+ this.accSerializer = accSerializer;
+ return self();
+ }
+
+ public T globalAggregate(
+ GeneratedNamespaceAggsHandleFunction<W>
localGeneratedAggregateFunction,
+ GeneratedNamespaceAggsHandleFunction<W>
globalGeneratedAggregateFunction,
+ GeneratedNamespaceAggsHandleFunction<W>
stateGeneratedAggregateFunction,
+ AbstractRowDataSerializer<RowData> accSerializer) {
+ this.localGeneratedAggregateFunction = localGeneratedAggregateFunction;
+ this.globalGeneratedAggregateFunction =
globalGeneratedAggregateFunction;
+ this.generatedAggregateFunction = stateGeneratedAggregateFunction;
+ this.accSerializer = accSerializer;
+ return self();
+ }
+
+ public AbstractWindowOperator<RowData, ?> build() {
+ checkNotNull(inputSerializer);
+ checkNotNull(keySerializer);
+ checkNotNull(shiftTimeZone);
+ return buildInner();
+ }
+
+ protected abstract AbstractWindowOperator<RowData, ?> buildInner();
Review Comment:
What's the usage here? Can we delay this part when introducing session
window?
##########
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java:
##########
@@ -193,7 +194,7 @@ public void open() throws Exception {
windowProperty = new GenericRowData(namedProperties.length);
windowAggResult = new JoinedRowData();
- WindowContext windowContext = new WindowContext();
+ PhthonWindowContext windowContext = new PhthonWindowContext();
Review Comment:
Phthon?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/MergeCallback.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import javax.annotation.Nullable;
+
+/**
+ * Callback to be used in when merging slices or windows for specifying which
slices or windows
+ * should be merged.
+ *
+ * @param <W> The type {@link Window} for windows or the type {@link Long} for
slices that this
+ * callback used to merge.
+ * @param <R> The result type like {@link java.util.Collection} or {@link
Iterable} to specify which
+ * slices or windows should be merged. TODO use {@link
java.util.Collection} uniformly.
+ */
+public interface MergeCallback<W, R> {
+
+ /**
+ * Specifies that states of the given windows or slices should be merged
into the result window
+ * or slice.
+ *
+ * @param mergeResult The resulting merged window or slice, {@code null}
if it represents a
+ * non-state namespace.
+ * @param toBeMerged Windows or slices that should be merged into one
window or slice.
+ */
+ void merge(@Nullable W mergeResult, R toBeMerged) throws Exception;
Review Comment:
BTW, the API is confusing... merge method doesn't return a result.
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/SliceAttachedWindowingStrategy.java:
##########
@@ -18,7 +18,7 @@
package org.apache.flink.table.planner.plan.logical;
-import
org.apache.flink.table.runtime.operators.window.slicing.SlicingWindowOperator;
+import
org.apache.flink.table.runtime.operators.window.windowtvf.slicing.SlicingWindowOperator;
Review Comment:
windowtvf?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/windowtvf/common/AbstractWindowOperator.java:
##########
@@ -73,34 +68,18 @@
* <p>Because aligned windows can be divided into finite number of
non-overlapping chunks (a.k.a.
* slices), which can apply efficient processing to share intermediate results.
*
- * <h3>Concept of Slice</h3>
- *
- * <p>Dividing a window of aligned windows into a finite number of
non-overlapping chunks, where the
- * chunks are slices. It has the following properties:
- *
- * <ul>
- * <li>An element must only belong to a single slice.
- * <li>Slices are non-overlapping, i.e. S_i and S_j should not have any
shared elements if i != j.
- * <li>A window is consist of a finite number of slices.
- * </ul>
- *
- * <h3>Abstraction of Slicing Window Operator</h3>
- *
- * <p>A slicing window operator is a simple wrap of {@link
SlicingWindowProcessor}. It delegates all
- * the important methods to the underlying processor, where the processor can
have different
- * implementation for aggregate and topk or others.
+ * <p>Note: currently, {@link AbstractWindowOperator} doesn't support
early-fire and late-arrival.
+ * Thus late elements (elements belong to emitted windows) will be simply
dropped.
*
- * <p>A {@link SlicingWindowProcessor} usually leverages the {@link
SliceAssigner} to assign slices
- * and calculate based on the slices. See {@link
SliceSharedWindowAggProcessor} as an example.
+ * <p>See more details in {@link
+ *
org.apache.flink.table.runtime.operators.window.windowtvf.slicing.SlicingWindowOperator}.
*
- * <p>Note: since {@link SlicingWindowProcessor} leverages slicing
optimization for aligned windows,
- * therefore, it doesn't support unaligned windows, e.g. session window.
+ * <p>TODO support unslicing window operator.
Review Comment:
unslicing -> unaligned
use jira issue id here
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/builder/AbstractWindowAggOperatorBuilder.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.builder;
+
+import org.apache.flink.table.data.RowData;
+import
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import
org.apache.flink.table.runtime.operators.window.windowtvf.common.AbstractWindowOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+
+import java.time.ZoneId;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The {@link AbstractWindowAggOperatorBuilder} is a base class for building
window aggregate
+ * operators.
+ *
+ * <p>See more details in {@link SlicingWindowAggOperatorBuilder}.
+ *
+ * <p>TODO support UnslicingWindowAggOperatorBuilder.
+ *
+ * @param <W> The type of the window. {@link Long} for slicing window.
+ * @param <T> The implementation of the abstract builder.
+ */
+public abstract class AbstractWindowAggOperatorBuilder<
+ W, T extends AbstractWindowAggOperatorBuilder> {
+
+ protected AbstractRowDataSerializer<RowData> inputSerializer;
+ protected PagedTypeSerializer<RowData> keySerializer;
+ protected AbstractRowDataSerializer<RowData> accSerializer;
+ protected GeneratedNamespaceAggsHandleFunction<W>
generatedAggregateFunction;
+ protected GeneratedNamespaceAggsHandleFunction<W>
localGeneratedAggregateFunction;
+ protected GeneratedNamespaceAggsHandleFunction<W>
globalGeneratedAggregateFunction;
+ protected ZoneId shiftTimeZone;
+
+ public T inputSerializer(AbstractRowDataSerializer<RowData>
inputSerializer) {
+ this.inputSerializer = inputSerializer;
+ return self();
+ }
+
+ public T shiftTimeZone(ZoneId shiftTimeZone) {
+ this.shiftTimeZone = shiftTimeZone;
+ return self();
+ }
+
+ public T keySerializer(PagedTypeSerializer<RowData> keySerializer) {
+ this.keySerializer = keySerializer;
+ return self();
Review Comment:
return this?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/GlobalAggCombiner.java:
##########
@@ -25,11 +25,11 @@
import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
import
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
-import
org.apache.flink.table.runtime.operators.window.combines.RecordsCombiner;
-import
org.apache.flink.table.runtime.operators.window.slicing.WindowTimerService;
-import org.apache.flink.table.runtime.operators.window.state.StateKeyContext;
-import org.apache.flink.table.runtime.operators.window.state.WindowState;
-import org.apache.flink.table.runtime.operators.window.state.WindowValueState;
+import
org.apache.flink.table.runtime.operators.window.windowtvf.combines.RecordsCombiner;
+import
org.apache.flink.table.runtime.operators.window.windowtvf.common.WindowTimerService;
Review Comment:
windowtvf -> tvf ?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/windowtvf/common/WindowAssigner.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.windowtvf.common;
+
+import
org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner;
+
+import java.io.Serializable;
+
+/**
+ * WindowAssigner is used to assign windows to elements.
+ *
+ * <p>The differences between {@link WindowAssigner} and {@link
GroupWindowAssigner} is that, this
+ * window assigner is translated from the new window TVF syntax, but the other
is from the legacy
+ * GROUP WINDOW FUNCTION syntax. In the long future, {@link
GroupWindowAssigner} will be dropped.
+ *
+ * <p>See more details in {@link AbstractWindowOperator}.
+ *
+ * <p>TODO support UnsliceAssigner.
+ */
+public interface WindowAssigner extends Serializable {
+
+ /**
+ * Returns {@code true} if elements are assigned to windows based on event
time, {@code false}
+ * based on processing time.
+ */
+ boolean isEventTime();
Review Comment:
Window Assigner doesn't care about event time or processing time? It assign
the element according to its time.
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/groupwindow/context/WindowContext.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.groupwindow.context;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.window.Window;
+import
org.apache.flink.table.runtime.operators.window.groupwindow.internal.InternalWindowProcessFunction;
+import
org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger;
+
+import java.time.ZoneId;
+import java.util.Collection;
+
+/** A context contains some information used for {@link
InternalWindowProcessFunction}. */
+public interface WindowContext<K, W extends Window> {
Review Comment:
Do we need generic type for key?
--
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]