[ 
https://issues.apache.org/jira/browse/FLINK-5658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15880012#comment-15880012
 ] 

ASF GitHub Bot commented on FLINK-5658:
---------------------------------------

Github user hongyuhong commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3386#discussion_r102650085
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala
 ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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.plan.nodes.datastream
    +
    +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.api.java.tuple.Tuple
    +import org.apache.flink.types.Row
    +import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
    +import FlinkRelBuilder.NamedWindowProperty
    +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
    +import org.apache.flink.table.runtime.aggregate._
    +import org.apache.flink.streaming.api.datastream.{AllWindowedStream, 
DataStream, WindowedStream}
    +import org.apache.flink.streaming.api.windowing.assigners._
    +import org.apache.flink.streaming.api.windowing.windows.{Window => 
DataStreamWindow}
    +import org.apache.flink.table.api.StreamTableEnvironment
    +import org.apache.flink.table.plan.nodes.CommonAggregate
    +
    +class DataStreamSlideEventTimeRowAgg(
    +    namedProperties: Seq[NamedWindowProperty],
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
    +    rowRelDataType: RelDataType,
    +    inputType: RelDataType,
    +    grouping: Array[Int])
    +  extends SingleRel(cluster, traitSet, inputNode)
    +  with CommonAggregate
    +  with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = rowRelDataType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
    +    new DataStreamSlideEventTimeRowAgg(
    +      namedProperties,
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      namedAggregates,
    +      getRowType,
    +      inputType,
    +      grouping)
    +  }
    +
    +  override def toString: String = {
    +    val inputFields = new Array[Int](inputType.getFieldCount)
    +    for (i <- 0 until inputType.getFieldCount)
    +      inputFields(i) = i
    +    s"Aggregate(${
    +      if (!grouping.isEmpty) {
    +        s"groupBy: (${groupingToString(inputType, grouping)}), "
    +      } else {
    +        ""
    +      }
    +    } orderBy: (eventtime), window: (unbounded), " +
    +      s"select: (${
    +        aggregationToString(
    +          inputType,
    +          inputFields,
    +          getRowType,
    +          namedAggregates,
    +          namedProperties)
    +      }))"
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    val inputFields = new Array[Int](inputType.getFieldCount)
    +    for (i <- 0 until inputType.getFieldCount)
    +      inputFields(i) = i
    +    super.explainTerms(pw)
    +      .itemIf("groupBy", groupingToString(inputType, grouping), 
!grouping.isEmpty)
    +      .item("orderBy", "eventtime")
    +      .item("window", "unbounded")
    +      .item("select", aggregationToString(
    +        inputType,
    +        inputFields,
    +        getRowType,
    +        namedAggregates,
    +        namedProperties))
    +  }
    +
    +  override def translateToPlan(tableEnv: StreamTableEnvironment): 
DataStream[Row] = {
    +
    +    val config = tableEnv.getConfig
    +    val groupingKeys = grouping.toArray
    +    val inputDS = 
input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
    +
    +    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
    +    val inputFields = new Array[Int](inputType.getFieldCount)
    --- End diff --
    
    Yes, it's much better, i will update it.


> Add event time OVER RANGE BETWEEN UNBOUNDED PRECEDING aggregation to SQL
> ------------------------------------------------------------------------
>
>                 Key: FLINK-5658
>                 URL: https://issues.apache.org/jira/browse/FLINK-5658
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: Yuhong Hong
>
> The goal of this issue is to add support for OVER RANGE aggregations on event 
> time streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT 
>   a, 
>   SUM(b) OVER (PARTITION BY c ORDER BY rowTime() RANGE BETWEEN UNBOUNDED 
> PRECEDING AND CURRENT ROW) AS sumB,
>   MIN(b) OVER (PARTITION BY c ORDER BY rowTime() RANGE BETWEEN UNBOUNDED 
> PRECEDING AND CURRENT ROW) AS minB
> FROM myStream
> {code}
> The following restrictions should initially apply:
> - All OVER clauses in the same SELECT clause must be exactly the same.
> - The PARTITION BY clause is optional (no partitioning results in single 
> threaded execution).
> - The ORDER BY clause may only have rowTime() as parameter. rowTime() is a 
> parameterless scalar function that just indicates processing time mode.
> - bounded PRECEDING is not supported (see FLINK-5655)
> - FOLLOWING is not supported.
> The restrictions will be resolved in follow up issues. If we find that some 
> of the restrictions are trivial to address, we can add the functionality in 
> this issue as well.
> This issue includes:
> - Design of the DataStream operator to compute OVER ROW aggregates
> - Translation from Calcite's RelNode representation (LogicalProject with 
> RexOver expression).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to