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



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedWindowAggregateRule.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.rules.physical.stream;
+
+import org.apache.flink.table.api.TableConfig;
+import 
org.apache.flink.table.planner.plan.logical.SliceAttachedWindowingStrategy;
+import 
org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
+import 
org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrategy;
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGlobalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLocalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.rules.physical.FlinkExpandConversionRule;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistributionTraitDef;
+import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait;
+import org.apache.flink.table.planner.plan.trait.UpdateKindTrait;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.utils.AggregatePhaseStrategy;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+
+import java.util.stream.IntStream;
+
+import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext;
+import static 
org.apache.flink.table.planner.utils.TableConfigUtils.getAggPhaseStrategy;
+
+/**
+ * Rule that matches {@link StreamPhysicalWindowAggregate} on {@link 
StreamPhysicalExchange} with
+ * following condition:
+ *
+ * <ul>
+ *   <li>the applied windowing is not on processing-time, because 
processing-time should be
+ *       materialized in a single node.
+ *   <li>two-phase aggregation is enabled in given TableConfig.
+ *   <li>all aggregate functions support merge() method.
+ *   <li>the input of exchange does not satisfy the shuffle distribution
+ * </ul>
+ *
+ * <p>This optimization is as known as local-global optimization for reducing 
data-shuffling. It
+ * splits window aggregation into two-stage window aggregation, i.e. 
local-aggregation and
+ * global-aggregation. The local-aggregation produces a partial result for 
each group and window
+ * slice before shuffle in stage 1, and then the partially aggregated results 
are shuffled by group
+ * key to global-aggregation which produces the final result in stage 2.
+ */
+public class TwoStageOptimizedWindowAggregateRule extends RelOptRule {
+
+    public static final TwoStageOptimizedWindowAggregateRule INSTANCE =
+            new TwoStageOptimizedWindowAggregateRule();
+
+    private TwoStageOptimizedWindowAggregateRule() {
+        super(
+                operand(
+                        StreamPhysicalWindowAggregate.class,
+                        operand(StreamPhysicalExchange.class, 
operand(RelNode.class, any()))),
+                "TwoStageOptimizedWindowAggregateRule");
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+        final StreamPhysicalWindowAggregate windowAgg = call.rel(0);
+        final RelNode realInput = call.rel(2);
+        final TableConfig tableConfig = 
unwrapContext(call.getPlanner()).getTableConfig();
+        final WindowingStrategy windowing = windowAgg.windowing();
+
+        // the two-phase optimization must be enabled
+        if (getAggPhaseStrategy(tableConfig) == 
AggregatePhaseStrategy.ONE_PHASE) {
+            return false;
+        }
+
+        // processing time window doesn't support two-phase,
+        // otherwise the processing-time can't be materialized in a single node
+        if (!windowing.isRowtime()) {
+            return false;
+        }
+
+        // all aggregate function should support merge() method
+        if 
(!AggregateUtil.doAllSupportPartialMerge(windowAgg.aggInfoList().aggInfos())) {
+            return false;
+        }
+
+        return !isInputSatisfyRequiredDistribution(realInput, 
windowAgg.grouping());
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final StreamPhysicalWindowAggregate windowAgg = call.rel(0);
+        final RelNode realInput = call.rel(2);
+        final WindowingStrategy windowing = windowAgg.windowing();
+
+        RelTraitSet localTraitSet =
+                realInput
+                        .getTraitSet()
+                        .plus(ModifyKindSetTrait.INSERT_ONLY())
+                        .plus(UpdateKindTrait.NONE());
+        StreamPhysicalLocalWindowAggregate localAgg =
+                new StreamPhysicalLocalWindowAggregate(
+                        windowAgg.getCluster(),
+                        localTraitSet,
+                        realInput,
+                        windowAgg.grouping(),
+                        windowAgg.aggCalls(),
+                        windowing);
+
+        // grouping keys is forwarded by local agg, use indices instead of 
groupings
+        int[] globalGrouping = IntStream.range(0, 
windowAgg.grouping().length).toArray();
+        FlinkRelDistribution globalDistribution = 
createDistribution(globalGrouping);
+        // create exchange if needed
+        RelNode newInput =
+                FlinkExpandConversionRule.satisfyDistribution(
+                        FlinkConventions.STREAM_PHYSICAL(), localAgg, 
globalDistribution);
+        RelTraitSet globalAggProvidedTraitSet = windowAgg.getTraitSet();
+
+        // we put sliceEnd at the end of local output fields
+        int sliceEndIndex = localAgg.getRowType().getFieldCount() - 1;

Review comment:
       I'm not sure whether `sliceEndIndex` is proper field name now, because 
it could be window end or slice end. 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.table.api.TableException;
+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.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+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.types.logical.LogicalType;
+
+import java.time.Duration;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The base class for window aggregate {@link ExecNode}. */
+public abstract class StreamExecWindowAggregateBase extends 
StreamExecAggregateBase {
+
+    protected StreamExecWindowAggregateBase(
+            int id,
+            List<InputProperty> inputProperties,
+            LogicalType outputType,
+            String description) {
+        super(id, inputProperties, outputType, description);
+        checkArgument(inputProperties.size() == 1);
+    }
+
+    // 
------------------------------------------------------------------------------------------
+    // Utilities
+    // 
------------------------------------------------------------------------------------------
+
+    protected SliceAssigner createSliceAssigner(WindowingStrategy 
windowingStrategy) {
+        WindowSpec windowSpec = windowingStrategy.getWindow();
+        if (windowingStrategy instanceof WindowAttachedWindowingStrategy) {
+            int windowEndIndex =
+                    ((WindowAttachedWindowingStrategy) 
windowingStrategy).getWindowEnd();
+            // 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);
+
+        } else if (windowingStrategy instanceof 
TimeAttributeWindowingStrategy) {
+            final int timeAttributeIndex;
+            if (windowingStrategy.isRowtime()) {
+                timeAttributeIndex =
+                        ((TimeAttributeWindowingStrategy) windowingStrategy)
+                                .getTimeAttributeIndex();
+            } else {
+                timeAttributeIndex = -1;
+            }
+            return createSliceAssigner(windowSpec, timeAttributeIndex);
+
+        } else {
+            throw new UnsupportedOperationException(windowingStrategy + " is 
not supported yet.");
+        }
+    }
+
+    protected SliceAssigner createSliceAssigner(WindowSpec windowSpec, int 
timeAttributeIndex) {
+        if (windowSpec instanceof TumblingWindowSpec) {
+            Duration size = ((TumblingWindowSpec) windowSpec).getSize();
+            return SliceAssigners.tumbling(timeAttributeIndex, size);
+
+        } else if (windowSpec instanceof HoppingWindowSpec) {

Review comment:
       Maybe we could move those check codes to `isValid` method in 
`StreamPhysicalWindowAggregate`?
   BTW, Currently, some validate code exists in  `isValid` method of 
`StreamPhysical**Node`, others exists in `StreamExec**Node. I'm a littler 
confused, is there any rule to apply when we decides where to validate node? 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.expressions._
+import 
org.apache.flink.table.planner.plan.logical.{SliceAttachedWindowingStrategy, 
WindowAttachedWindowingStrategy, WindowingStrategy}
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalWindowAggregate
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
+import 
org.apache.flink.table.planner.plan.rules.physical.stream.TwoStageOptimizedWindowAggregateRule
+import 
org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration
+import org.apache.flink.table.planner.plan.utils.{AggregateUtil, 
FlinkRelOptUtil, RelExplainUtil, WindowUtil}
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+/**
+ * Streaming global window aggregate physical node.
+ *
+ * <p>This is a global-aggregation node optimized from 
[[StreamPhysicalWindowAggregate]] after
+ * [[TwoStageOptimizedWindowAggregateRule]] optimization.
+ *
+ * <p>The windowing of global window aggregate must be 
[[SliceAttachedWindowingStrategy]] or
+ * [[WindowAttachedWindowingStrategy]] because windowing or slicing has been 
applied by
+ * local window aggregate. There is no time attribute and no window start 
columns on
+ * the output of local window aggregate, but slice end.
+ *
+ * @see [[TwoStageOptimizedWindowAggregateRule]]
+ * @see [[StreamPhysicalWindowAggregate]]
+ */
+class StreamPhysicalGlobalWindowAggregate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputRel: RelNode,
+    val inputRowTypeOfLocalAgg: RelDataType,
+    val grouping: Array[Int],
+    val aggCalls: Seq[AggregateCall],
+    val windowing: WindowingStrategy,
+    val namedWindowProperties: Seq[PlannerNamedWindowProperty])
+  extends SingleRel(cluster, traitSet, inputRel)
+  with StreamPhysicalRel {
+
+  private lazy val aggInfoList = AggregateUtil.deriveWindowAggregateInfoList(
+    FlinkTypeFactory.toLogicalRowType(inputRowTypeOfLocalAgg),
+    aggCalls,
+    windowing.getWindow,
+    isStateBackendDataViews = true)
+
+  override def requireWatermark: Boolean = windowing.isRowtime
+
+  override def deriveRowType(): RelDataType = {
+    WindowUtil.deriveWindowAggregateRowType(
+      grouping,
+      aggCalls,
+      windowing,
+      namedWindowProperties,
+      inputRel.getRowType,
+      cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory])
+  }
+
+  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,
+        getRowType,
+        aggInfoList,
+        grouping,
+        namedWindowProperties,
+        isGlobal = true))
+  }
+
+  override def copy(
+      traitSet: RelTraitSet,
+      inputs: util.List[RelNode]): RelNode = {
+    new StreamPhysicalGlobalWindowAggregate(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputRowTypeOfLocalAgg,
+      grouping,
+      aggCalls,
+      windowing,
+      namedWindowProperties
+    )
+  }
+
+  override def translateToExecNode(): ExecNode[_] = {
+    checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this))
+    if (!windowing.isInstanceOf[SliceAttachedWindowingStrategy] &&
+        !windowing.isInstanceOf[WindowAttachedWindowingStrategy]) {
+      throw new TableException("Global window aggregate should only accept " +

Review comment:
       Maybe we could move those check codes to `isValid` method?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala
##########
@@ -198,6 +204,79 @@ object WindowUtil {
     new TimeAttributeWindowingStrategy(windowSpec, timeAttributeType, 
timeIndex)
   }
 
+  /**
+   * Window TVF based aggregations don't support early-fire and late-fire,
+   * throws exception when the configurations are set.
+   */
+  def checkEmitConfiguration(tableConfig: TableConfig): Unit = {
+    val conf = tableConfig.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()}'.")
+    }
+  }
+
+  // 
------------------------------------------------------------------------------------------
+  // RelNode RowType
+  // 
------------------------------------------------------------------------------------------
+
+  def deriveWindowAggregateRowType(
+      grouping: Array[Int],
+      aggCalls: Seq[AggregateCall],
+      windowing: WindowingStrategy,
+      namedWindowProperties: Seq[PlannerNamedWindowProperty],
+      inputRowType: RelDataType,
+      typeFactory: FlinkTypeFactory): RelDataType = {
+    val groupSet = ImmutableBitSet.of(grouping: _*)
+    val baseType = Aggregate.deriveRowType(
+      typeFactory,
+      inputRowType,
+      false,
+      groupSet,
+      Collections.singletonList(groupSet),
+      aggCalls)
+    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.getProperty match {
+        case _: PlannerWindowStart | _: PlannerWindowEnd =>
+          
LogicalTypeUtils.removeTimeAttributes(windowing.getTimeAttributeType).copy(false)
+        case _: PlannerRowtimeAttribute | _: PlannerProctimeAttribute =>
+          windowing.getTimeAttributeType.copy(false)
+      }
+      builder.add(namedProp.getName, 
typeFactory.createFieldTypeFromLogicalType(timeType))
+    }
+    builder.build()
+  }
+
+  /**
+   * Derives output row type from local window aggregate
+   */
+  def deriveLocalWindowAggregateRowType(
+      aggInfoList: AggregateInfoList,
+      grouping: Array[Int],
+      inputRowType: RelDataType,
+      typeFactory: FlinkTypeFactory): RelDataType = {
+    val accTypes = aggInfoList.getAccTypes
+    val groupingTypes = grouping
+      .map(inputRowType.getFieldList.get(_).getType)
+      .map(FlinkTypeFactory.toLogicalType)
+    val sliceEndType = Array(DataTypes.BIGINT().getLogicalType)
+
+    val groupingNames = grouping.map(inputRowType.getFieldNames.get(_))
+    val accFieldNames = inferAggAccumulatorNames(aggInfoList)
+    val sliceEndName = Array("$slice_end")

Review comment:
       I'm not sure whether `slice_end` is proper field name now, because it 
could be window end or slice end. 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/logical/WindowAttachedWindowingStrategy.java
##########
@@ -52,13 +52,28 @@ public WindowAttachedWindowingStrategy(
         this.windowEnd = windowEnd;
     }
 
+    /**
+     * Creates a {@link WindowAttachedWindowingStrategy} which only {@link 
#windowEnd} is attatched.
+     */
+    public WindowAttachedWindowingStrategy(
+            WindowSpec window, LogicalType timeAttributeType, int windowEnd) {
+        super(window, timeAttributeType);
+        this.windowStart = -1;
+        this.windowEnd = windowEnd;
+    }
+
     @Override
     public String toSummaryString(String[] inputFieldNames) {
-        checkArgument(windowStart >= 0 && windowStart < 
inputFieldNames.length);
-        String windowing =
-                String.format(
-                        "win_start=[%s], win_end=[%s]",
-                        inputFieldNames[windowStart], 
inputFieldNames[windowEnd]);
+        checkArgument(windowEnd >= 0 && windowEnd < inputFieldNames.length);

Review comment:
       Maybe we could move the check code to constructor?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/logical/SliceAttachedWindowingStrategy.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.logical;
+
+import 
org.apache.flink.table.runtime.operators.window.slicing.SlicingWindowOperator;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * A windowing strategy that gets windows from input columns as window slice 
have been assigned and
+ * attached to the physical columns. The window slice is usually identified by 
slice end timestamp.
+ *
+ * @see SlicingWindowOperator for more details about which windows can apply 
slicing.
+ */
+@JsonTypeName("SliceAttached")
+public class SliceAttachedWindowingStrategy extends WindowingStrategy {
+    public static final String FIELD_NAME_SLICE_END = "sliceEnd";
+
+    @JsonProperty(FIELD_NAME_SLICE_END)
+    private final int sliceEnd;
+
+    @JsonCreator
+    public SliceAttachedWindowingStrategy(
+            @JsonProperty(FIELD_NAME_WINDOW) WindowSpec window,
+            @JsonProperty(value = FIELD_NAME_TIME_ATTRIBUTE_TYPE) LogicalType 
timeAttributeType,
+            @JsonProperty(FIELD_NAME_SLICE_END) int sliceEnd) {
+        super(window, timeAttributeType);
+        this.sliceEnd = sliceEnd;
+    }
+
+    @Override
+    public String toSummaryString(String[] inputFieldNames) {
+        checkArgument(sliceEnd >= 0 && sliceEnd < inputFieldNames.length);

Review comment:
       Maybe we could move the check code to constructor?




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