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

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

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

    https://github.com/apache/flink/pull/2938#discussion_r95554086
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala
 ---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.RichGroupReduceFunction
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.{Collector, Preconditions}
    +
    +import scala.collection.JavaConversions._
    +
    +
    +/**
    +  * It wraps the aggregate logic inside of
    +  * [[org.apache.flink.api.java.operators.GroupReduceOperator]].
    +  * It is only used for tumbling count-window on batch.
    +  *
    +  * @param windowSize Tumble count window size
    +  * @param aggregates The aggregate functions.
    +  * @param groupKeysMapping The index mapping of group keys between 
intermediate aggregate Row
    +  *                         and output Row.
    +  * @param aggregateMapping The index mapping between aggregate function 
list and aggregated value
    +  *                         index in output Row.
    +  * @param intermediateRowArity The intermediate row field count
    +  * @param finalRowArity The output row field count
    +  */
    +class DataSetTumbleCountWindowAggReduceGroupFunction(
    +    private val windowSize: Long,
    +    private val aggregates: Array[Aggregate[_ <: Any]],
    +    private val groupKeysMapping: Array[(Int, Int)],
    +    private val aggregateMapping: Array[(Int, Int)],
    +    private val intermediateRowArity: Int,
    +    private val finalRowArity: Int)
    +  extends RichGroupReduceFunction[Row, Row] {
    +
    +  private var aggregateBuffer: Row = _
    +  private var output: Row = _
    +
    +  override def open(config: Configuration) {
    +    Preconditions.checkNotNull(aggregates)
    +    Preconditions.checkNotNull(groupKeysMapping)
    +    aggregateBuffer = new Row(intermediateRowArity)
    +    output = new Row(finalRowArity)
    +  }
    +
    +  override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = 
{
    +
    +    var count: Long = 0
    +
    +    records.foreach( (record) => {
    --- End diff --
    
    Just a small comment here for future implementations. We should try to 
reduce Scala magic in runtime classes. I would use a good old while loop here. 
We don't know how many method calls and helper objects are created in this 
reducer.


> Add tumbling group-windows for batch tables
> -------------------------------------------
>
>                 Key: FLINK-4692
>                 URL: https://issues.apache.org/jira/browse/FLINK-4692
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Timo Walther
>            Assignee: Jark Wu
>
> Add Tumble 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