beyond1920 commented on a change in pull request #14905:
URL: https://github.com/apache/flink/pull/14905#discussion_r580098617



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SliceAssigners.java
##########
@@ -321,20 +322,15 @@ public void mergeSlices(long sliceEnd, MergeCallback 
callback) throws Exception
         private static final long serialVersionUID = 1L;
 
         private final int windowEndIndex;
-        private final long windowSize;
+        private final SliceAssigner innerAssigner;
         private final ReusableListIterable reuseExpiredList = new 
ReusableListIterable();
 
-        public WindowedSliceAssigner(int windowEndIndex, long windowSize) {
+        public WindowedSliceAssigner(int windowEndIndex, SliceAssigner 
innerAssigner) {

Review comment:
       `innerAssigner ` seems to only be used to calculate window start?
   Could we update constructor to  pass windowStartIndex and windowEndIndex 
because it's a little strange to create `SliceAssigner` based on 
`Integer.MAX_VALUE` as `timeAttributeIndex` for 
`WindowAttachedWindowingStrategy`,

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.planner.plan.nodes.exec.stream;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.expressions.PlannerWindowProperty;
+import org.apache.flink.table.planner.plan.logical.CumulativeWindowSpec;
+import org.apache.flink.table.planner.plan.logical.HoppingWindowSpec;
+import 
org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
+import org.apache.flink.table.planner.plan.logical.TumblingWindowSpec;
+import 
org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrategy;
+import org.apache.flink.table.planner.plan.logical.WindowSpec;
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.aggregate.window.SlicingWindowAggOperatorBuilder;
+import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigner;
+import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigners;
+import 
org.apache.flink.table.runtime.operators.window.slicing.SliceSharedAssigner;
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Stream {@link ExecNode} for window table-valued based aggregate.
+ *
+ * <p>The differences between {@link StreamExecWindowAggregate} and {@link
+ * StreamExecGroupWindowAggregate} is that, this node is translated from 
window TVF syntax, but the
+ * other is from the legacy GROUP WINDOW FUNCTION syntax. In the long future, 
{@link
+ * StreamExecGroupWindowAggregate} will be dropped.
+ */
+public class StreamExecWindowAggregate extends ExecNodeBase<RowData>
+        implements StreamExecNode<RowData> {
+
+    private static final long WINDOW_AGG_MEMORY_RATIO = 100;
+
+    private final int[] grouping;
+    private final AggregateCall[] aggCalls;
+    private final WindowingStrategy windowing;
+    private final PlannerNamedWindowProperty[] namedWindowProperties;
+
+    public StreamExecWindowAggregate(
+            int[] grouping,
+            AggregateCall[] aggCalls,
+            WindowingStrategy windowing,
+            PlannerNamedWindowProperty[] namedWindowProperties,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        super(Collections.singletonList(inputProperty), outputType, 
description);
+        this.grouping = grouping;
+        this.aggCalls = aggCalls;
+        this.windowing = windowing;
+        this.namedWindowProperties = namedWindowProperties;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(PlannerBase 
planner) {
+        final ExecEdge inputEdge = getInputEdges().get(0);
+        final Transformation<RowData> inputTransform =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputEdge.getOutputType();
+
+        final TableConfig config = planner.getTableConfig();
+        final SliceAssigner sliceAssigner = createSliceAssigner(windowing);
+
+        // Hopping window requires additional COUNT(*) to determine whether to 
register next timer
+        // through whether the current fired window is empty, see 
SliceSharedWindowAggProcessor.
+        final boolean needInputCount = sliceAssigner instanceof 
SliceAssigners.HoppingSliceAssigner;
+        final boolean[] aggCallNeedRetractions = new boolean[aggCalls.length];
+        Arrays.fill(aggCallNeedRetractions, false);
+        final AggregateInfoList aggInfoList =
+                AggregateUtil.transformToStreamAggregateInfoList(
+                        inputRowType,
+                        
JavaScalaConversionUtil.toScala(Arrays.asList(aggCalls)),
+                        aggCallNeedRetractions,
+                        needInputCount,
+                        true, // isStateBackendDataViews
+                        true); // needDistinctInfo
+
+        final GeneratedNamespaceAggsHandleFunction<Long> generatedAggsHandler =
+                createAggsHandler(
+                        sliceAssigner,
+                        aggInfoList,
+                        config,
+                        planner.getRelBuilder(),
+                        inputRowType.getChildren());
+
+        final LogicalType[] keyTypes =
+                Arrays.stream(grouping)
+                        .mapToObj(inputRowType::getTypeAt)
+                        .toArray(LogicalType[]::new);
+        final LogicalType[] accTypes = 
convertToLogicalTypes(aggInfoList.getAccTypes());
+
+        final OneInputStreamOperator<RowData, RowData> windowOperator =
+                SlicingWindowAggOperatorBuilder.builder()
+                        .inputType(inputRowType)
+                        .keyTypes(keyTypes)
+                        .assigner(sliceAssigner)
+                        .countStarIndex(aggInfoList.getIndexOfCountStar())
+                        .aggregate(generatedAggsHandler, accTypes)
+                        .build();
+
+        final OneInputTransformation<RowData, RowData> transform =
+                ExecNodeUtil.createOneInputTransformation(
+                        inputTransform,
+                        getDescription(),
+                        SimpleOperatorFactory.of(windowOperator),
+                        InternalTypeInfo.of(getOutputType()),
+                        inputTransform.getParallelism(),
+                        WINDOW_AGG_MEMORY_RATIO);
+
+        if (inputsContainSingleton()) {
+            transform.setParallelism(1);
+            transform.setMaxParallelism(1);
+        }
+
+        // set KeyType and Selector for state
+        final RowDataKeySelector selector =
+                KeySelectorUtil.getRowDataSelector(grouping, 
InternalTypeInfo.of(inputRowType));
+        transform.setStateKeySelector(selector);
+        transform.setStateKeyType(selector.getProducedType());
+        return transform;
+    }
+
+    private GeneratedNamespaceAggsHandleFunction<Long> createAggsHandler(
+            SliceAssigner sliceAssigner,
+            AggregateInfoList aggInfoList,
+            TableConfig config,
+            RelBuilder relBuilder,
+            List<LogicalType> fieldTypes) {
+        final AggsHandlerCodeGenerator generator =
+                new AggsHandlerCodeGenerator(
+                                new CodeGeneratorContext(config),
+                                relBuilder,
+                                JavaScalaConversionUtil.toScala(fieldTypes),
+                                false) // copyInputField
+                        .needAccumulate();
+
+        if (sliceAssigner instanceof SliceSharedAssigner) {
+            generator.needMerge(0, false, null);
+        }
+
+        final List<PlannerWindowProperty> windowProperties =
+                Arrays.asList(
+                        Arrays.stream(namedWindowProperties)
+                                .map(PlannerNamedWindowProperty::property)
+                                .toArray(PlannerWindowProperty[]::new));
+
+        return generator.generateNamespaceAggsHandler(
+                "GroupingWindowAggsHandler",
+                aggInfoList,
+                JavaScalaConversionUtil.toScala(windowProperties),
+                sliceAssigner);
+    }
+
+    // 
------------------------------------------------------------------------------------------
+    // Utilities
+    // 
------------------------------------------------------------------------------------------
+
+    private static SliceAssigner createSliceAssigner(WindowingStrategy 
windowingStrategy) {
+        WindowSpec windowSpec = windowingStrategy.window();
+        if (windowingStrategy instanceof WindowAttachedWindowingStrategy) {
+            int windowEndIndex = ((WindowAttachedWindowingStrategy) 
windowingStrategy).windowEnd();
+            // we don't need time attribute to assign windows, use a magic 
value in this case
+            SliceAssigner innerAssigner = createSliceAssigner(windowSpec, 
Integer.MAX_VALUE);
+            return SliceAssigners.windowed(windowEndIndex, innerAssigner);

Review comment:
       It's a little strange to create SliceAssigner based on 
`Integer.MAX_VALUE` as timeAttributeIdx, could we just pass `windowStartIndex` 
and `windowEndIndex ` when call `SliceAssigners.windowed(`? 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.planner.plan.nodes.physical.stream
+
+import org.apache.flink.table.api.TableException
+import 
org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.expressions.{PlannerProctimeAttribute, 
PlannerRowtimeAttribute, PlannerWindowEnd, PlannerWindowStart}
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowAggregate
+import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, 
RelExplainUtil}
+import 
org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED,
 TABLE_EXEC_EMIT_LATE_FIRE_ENABLED}
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.{Aggregate, AggregateCall}
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.calcite.util.ImmutableBitSet
+
+import java.util
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+
+/**
+ * Streaming window aggregate physical node which will be translate to window 
aggregate operator.
+ *
+ * Note: The differences between [[StreamPhysicalWindowAggregate]] and
+ * [[StreamPhysicalGroupWindowAggregate]] is that, 
[[StreamPhysicalWindowAggregate]] is translated
+ * from window TVF syntax, but the other is from the legacy GROUP WINDOW 
FUNCTION syntax.
+ * In the long future, [[StreamPhysicalGroupWindowAggregate]] will be dropped.
+ */
+class StreamPhysicalWindowAggregate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputRel: RelNode,
+    val grouping: Array[Int],
+    val aggCalls: Seq[AggregateCall],
+    val windowing: WindowingStrategy,
+    val namedWindowProperties: Seq[PlannerNamedWindowProperty])
+  extends SingleRel(cluster, traitSet, inputRel)
+  with StreamPhysicalRel {
+
+  override def requireWatermark: Boolean = windowing.isRowtime
+
+  override def deriveRowType(): RelDataType = {
+    val groupSet = ImmutableBitSet.of(grouping: _*)
+    val baseType = Aggregate.deriveRowType(
+      cluster.getTypeFactory,
+      getInput.getRowType,
+      false,
+      groupSet,
+      Collections.singletonList(groupSet),
+      aggCalls.asJava)
+    val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    val builder = typeFactory.builder
+    builder.addAll(baseType.getFieldList)
+    namedWindowProperties.foreach { namedProp =>
+      // use types from windowing strategy which keeps the precision and 
timestamp type
+      // cast the type to not null type, because window properties should 
never be null
+      val timeType = namedProp.property match {
+        case PlannerWindowStart(_) | PlannerWindowEnd(_) =>
+          
LogicalTypeUtils.removeTimeAttributes(windowing.timeAttributeType).copy(false)
+        case PlannerRowtimeAttribute(_) | PlannerProctimeAttribute(_) =>
+          windowing.timeAttributeType.copy(false)
+      }
+      builder.add(namedProp.name, 
typeFactory.createFieldTypeFromLogicalType(timeType))
+    }
+    builder.build()
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    val inputRowType = getInput.getRowType
+    val inputFieldNames = inputRowType.getFieldNames.asScala.toArray
+    super.explainTerms(pw)
+      .itemIf("groupBy", RelExplainUtil.fieldToString(grouping, inputRowType), 
grouping.nonEmpty)
+      .item("window", windowing.toSummaryString(inputFieldNames))
+      .item("select", RelExplainUtil.streamWindowAggregationToString(
+        inputRowType,
+        grouping,
+        getRowType,
+        aggCalls,
+        namedWindowProperties))
+  }
+
+  override def copy(
+      traitSet: RelTraitSet,
+      inputs: util.List[RelNode]): RelNode = {
+    new StreamPhysicalWindowAggregate(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      grouping,
+      aggCalls,
+      windowing,
+      namedWindowProperties
+    )
+  }
+
+  override def translateToExecNode(): ExecNode[_] = {
+    val conf = FlinkRelOptUtil.getTableConfigFromContext(this).getConfiguration
+    if (conf.getBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED) ||
+      conf.getBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)) {
+      throw new TableException("Currently, window table function based 
aggregate doesn't " +
+        s"support early-fire and late-fire configuration " +
+        s"'${TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED.key()}' and " +
+        s"'${TABLE_EXEC_EMIT_LATE_FIRE_ENABLED.key()}'.")
+    }
+    new StreamExecWindowAggregate(

Review comment:
       It's better to add check for allow lateness it seems not to be supported 
yet.

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.planner.plan.nodes.exec.stream;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.expressions.PlannerWindowProperty;
+import org.apache.flink.table.planner.plan.logical.CumulativeWindowSpec;
+import org.apache.flink.table.planner.plan.logical.HoppingWindowSpec;
+import 
org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
+import org.apache.flink.table.planner.plan.logical.TumblingWindowSpec;
+import 
org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrategy;
+import org.apache.flink.table.planner.plan.logical.WindowSpec;
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.aggregate.window.SlicingWindowAggOperatorBuilder;
+import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigner;
+import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigners;
+import 
org.apache.flink.table.runtime.operators.window.slicing.SliceSharedAssigner;
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.tools.RelBuilder;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Stream {@link ExecNode} for window table-valued based aggregate.
+ *
+ * <p>The differences between {@link StreamExecWindowAggregate} and {@link
+ * StreamExecGroupWindowAggregate} is that, this node is translated from 
window TVF syntax, but the
+ * other is from the legacy GROUP WINDOW FUNCTION syntax. In the long future, 
{@link
+ * StreamExecGroupWindowAggregate} will be dropped.
+ */
+public class StreamExecWindowAggregate extends ExecNodeBase<RowData>
+        implements StreamExecNode<RowData> {
+
+    private static final long WINDOW_AGG_MEMORY_RATIO = 100;
+
+    private final int[] grouping;
+    private final AggregateCall[] aggCalls;
+    private final WindowingStrategy windowing;
+    private final PlannerNamedWindowProperty[] namedWindowProperties;
+
+    public StreamExecWindowAggregate(
+            int[] grouping,
+            AggregateCall[] aggCalls,
+            WindowingStrategy windowing,
+            PlannerNamedWindowProperty[] namedWindowProperties,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        super(Collections.singletonList(inputProperty), outputType, 
description);
+        this.grouping = grouping;
+        this.aggCalls = aggCalls;
+        this.windowing = windowing;
+        this.namedWindowProperties = namedWindowProperties;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(PlannerBase 
planner) {
+        final ExecEdge inputEdge = getInputEdges().get(0);
+        final Transformation<RowData> inputTransform =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        final RowType inputRowType = (RowType) inputEdge.getOutputType();
+
+        final TableConfig config = planner.getTableConfig();
+        final SliceAssigner sliceAssigner = createSliceAssigner(windowing);
+
+        // Hopping window requires additional COUNT(*) to determine whether to 
register next timer
+        // through whether the current fired window is empty, see 
SliceSharedWindowAggProcessor.
+        final boolean needInputCount = sliceAssigner instanceof 
SliceAssigners.HoppingSliceAssigner;

Review comment:
       Could you please explain why need COUNT(*) for HoppingSliceAssigner in 
detail, I'm still confused about this. Thanks a lot.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to