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

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

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

    https://github.com/apache/flink/pull/3150#discussion_r96605812
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregateCombineGroupFunction.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.runtime.aggregate
    +
    +import java.lang.Iterable
    +
    +import org.apache.flink.api.common.functions.RichGroupCombineFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable
    +import org.apache.flink.types.Row
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.util.{Collector, Preconditions}
    +
    +import scala.collection.JavaConversions._
    +
    +/**
    +  * This wraps the aggregate logic inside of
    +  * [[org.apache.flink.api.java.operators.GroupCombineOperator]].
    +  *
    +  * @param aggregates The aggregate functions.
    +  * @param groupingKeys
    +  * @param intermediateRowArity The intermediate row field count.
    +  * @param gap  Session time window gap.
    +  * @param intermediateRowType Intermediate row data type.
    +  */
    +class DataSetSessionWindowAggregateCombineGroupFunction(
    +    aggregates: Array[Aggregate[_ <: Any]],
    +    groupingKeys: Array[Int],
    +    intermediateRowArity: Int,
    +    gap: Long,
    +    @transient intermediateRowType: TypeInformation[Row])
    +  extends RichGroupCombineFunction[Row,Row] with ResultTypeQueryable[Row] {
    +
    +  private var aggregateBuffer: Row = _
    +  private var rowTimePos = 0
    +
    +  override def open(config: Configuration) {
    +    Preconditions.checkNotNull(aggregates)
    +    Preconditions.checkNotNull(groupingKeys)
    +    aggregateBuffer = new Row(intermediateRowArity)
    +    rowTimePos = intermediateRowArity - 2
    +  }
    +
    +  /**
    +    * For sub-grouped intermediate aggregate Rows, divide window based on 
the row-time
    +    * (current'row-time - previous’row-time > gap), and then merge data 
(within a unified window)
    +    * into an aggregate buffer.
    +    *
    +    * @param records  Sub-grouped intermediate aggregate Rows .
    +    * @return Combined intermediate aggregate Row.
    +    *
    +    */
    +  override def combine(
    +    records: Iterable[Row],
    +    out: Collector[Row]): Unit = {
    +
    +    var head:Row = null
    +    var lastRowTime: Option[Long] = None
    +    var currentRowTime: Option[Long] = None
    +
    +    records.foreach(
    --- End diff --
    
    We should avoid Scala magic in runtime classes as much as possible. Because 
we don't know the runtime behavior. Can you replace this by a while loop?


> Add session group-windows for batch tables    
> -------------------------------------------
>
>                 Key: FLINK-4693
>                 URL: https://issues.apache.org/jira/browse/FLINK-4693
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Timo Walther
>            Assignee: sunjincheng
>
> Add Session group-windows for batch tables as described in 
> [FLIP-11|https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations].
>  



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to