You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by sunjincheng121 <gi...@git.apache.org> on 2017/06/21 13:24:52 UTC

[GitHub] flink pull request #4157: [Flink 6649][table]Improve Non-window group aggreg...

GitHub user sunjincheng121 opened a pull request:

    https://github.com/apache/flink/pull/4157

    [Flink 6649][table]Improve Non-window group aggregate with update int…

    In this PR. I have add supports updating the calculated data according to the specified time interval on non-window group AGG. If we config the time interval is N seconds, then the next update time relative to the latest update time T, is T+N seconds. For example, the time interval is 2 seconds, the previous update time is T seconds, and the next update time T1> = T + 2 seconds. If no data arrives during T to T + 2, no updates are made.
    
    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [x] General
      - The pull request references the related JIRA issue ("[Flink 6649][table]Improve Non-window group aggregate with update interval.")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sunjincheng121/flink FLINK-6649

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4157.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4157
    
----
commit 68ecd416bbdf70ed01d195c805c92d03f5ae6004
Author: sunjincheng121 <su...@gmail.com>
Date:   2017-06-14T01:36:48Z

    [Flink 6649][table]Improve Non-window group aggregate with update interval.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125423783
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    --- End diff --
    
    How about `IntervalUpdateGroupAgg`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125289222
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    --- End diff --
    
    Registering a state for `Row` without a `RowTypeInfo` is inefficient, because it will use the `KryoSerializer`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r126541801
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (currentTime >= triggerTimer) {
    +
    +      val newTimer = currentTime + queryConfig.getUnboundedAggregateUpdateInterval
    +
    +      emitTimerState.update(newTimer)
    +
    +      ctx.timerService().registerProcessingTimeTimer(newTimer)
    --- End diff --
    
    I think you are right. For now, a processing-time timer is sufficient and maybe we don't need an event-time timer at all. This parameter does not affect the semantics of the result, so processing time is fine and easier to reason about. I also agree that a single configuration parameter is enough. If `updateInterval` is not set or set to `0`, results are immediately updated. Otherwise, according to the specified interval.
    
    However, I still think that we need three different implementations of the operator:
    1. Immediate update (the one that we already have)
    2. Interval update for processing time, i.e., no watermarks that need to be hold back
    3. Interval update for event time, i.e., holding back watermarks. The updates should always have the watermark of the last update, which means we have to hold back the watermarks. This operator cannot be implemented with a `ProcessFunction` and requires a custom operator.
    
    The cases 2. and 3. are determined based on the time characteristics setting of the `StreamExecutionEnvironment`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125281365
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala ---
    @@ -37,6 +37,13 @@ class BatchQueryConfig private[table] extends QueryConfig
     class StreamQueryConfig private[table] extends QueryConfig {
     
       /**
    +    * The non-windowed groupby aggregate update the calculation result according a configuration of
    +    * time interval. By default non-windowed groupby aggregate will update calculation result each
    +    * row.
    +    */
    +  private var unboundedAggregateUpdateInterval: Time = Time.milliseconds(0)
    --- End diff --
    
    I would not make this parameter specific for unbounded aggregates but use it for all types of (supported) operators. The output of a query will always be synced to the slowest update rate.
    
    Hence I propose to rename the parameter to `updateRate` or `updateInterval`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125299675
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (currentTime >= triggerTimer) {
    +
    +      val newTimer = currentTime + queryConfig.getUnboundedAggregateUpdateInterval
    +
    +      emitTimerState.update(newTimer)
    +
    +      ctx.timerService().registerProcessingTimeTimer(newTimer)
    --- End diff --
    
    Should we define update rate on processing time or event time?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125322758
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala ---
    @@ -80,18 +80,79 @@ class NonWindowHarnessTest extends HarnessTestBase {
     
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
    -
    -    verify(expectedOutput, result, new RowResultSortComparator(6))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, "bbb", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, "aaa", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, "aaa", 6: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, "aaa", 10: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, "bbb", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, "aaa", 5: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, "aaa", 11: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, "aaa", 18: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, "bbb", 3: JInt), true), 1))
    +
    +    verifySorted(expectedOutput, result, new RowResultSortComparator)
    +
    +    testHarness.close()
    +  }
    +
    +  @Test
    +  def testProcTimeNonWindowWithUpdateInterval(): Unit = {
    +
    +    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
    +      new GroupAggProcessFunctionWithUpdateInterval(
    +        genSumAggFunction,
    +        sumAggregationStateType,
    +        sumAggregationRowType,
    +        false,
    +        queryConfig
    +        .withIdleStateRetentionTime(Time.seconds(4), Time.seconds(5))
    +        .withUnboundedAggregateUpdateInterval(Time.seconds(1))))
    +
    +    val testHarness =
    +      createHarnessTester(
    +        processFunction,
    +        new TupleRowKeySelector[String](2),
    +        BasicTypeInfo.STRING_TYPE_INFO)
    +
    +    testHarness.open()
    +
    +    testHarness.setProcessingTime(1)
    +
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
    +    testHarness.setProcessingTime(1000)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
    +
    +    testHarness.setProcessingTime(1002)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(4003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
    +
    +    // clear all states
    +    testHarness.setProcessingTime(10003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(12003)
    +
    +    val result = testHarness.getOutput
    +
    +    val expectedOutput = new ConcurrentLinkedQueue[Object]()
    +
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 6: JInt), true), 1001))
    --- End diff --
    
    The `ProcessingFunction` sets the timestamp of the processing time timer as event-time to all records emitted in `onTimer()`. IMO, this is pretty bad behavior because processing time and event time (and existing watermarks) are not related at all. I think the best behavior would be to assign the timestamp of the last updating record. However, this would again require to hold back watermarks which is not trivial (and can only be done with a custom operator).
    
    For now, I would propose to set the current watermark as timestamp such that records are not considered as late data. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125423153
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala ---
    @@ -37,6 +37,13 @@ class BatchQueryConfig private[table] extends QueryConfig
     class StreamQueryConfig private[table] extends QueryConfig {
     
       /**
    +    * The non-windowed groupby aggregate update the calculation result according a configuration of
    +    * time interval. By default non-windowed groupby aggregate will update calculation result each
    +    * row.
    +    */
    +  private var unboundedAggregateUpdateInterval: Time = Time.milliseconds(0)
    --- End diff --
    
    I think `updateRate` meant update result in a fixed frequency, so we should register a process-time timer. i.e., System clock driven the trigger.  And `updateInterval` meant update result according coming data. so I we should register a event-time timer. i.e, Data driven the trigger.
    I think it again, I suggest using `updateInterval`, i.e., we emit result according event-time of data.
    What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125316505
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala ---
    @@ -80,18 +80,79 @@ class NonWindowHarnessTest extends HarnessTestBase {
     
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
    -
    -    verify(expectedOutput, result, new RowResultSortComparator(6))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, "bbb", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, "aaa", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, "aaa", 6: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, "aaa", 10: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, "bbb", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, "aaa", 5: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, "aaa", 11: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, "aaa", 18: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, "bbb", 3: JInt), true), 1))
    +
    +    verifySorted(expectedOutput, result, new RowResultSortComparator)
    +
    +    testHarness.close()
    +  }
    +
    +  @Test
    +  def testProcTimeNonWindowWithUpdateInterval(): Unit = {
    +
    +    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
    +      new GroupAggProcessFunctionWithUpdateInterval(
    +        genSumAggFunction,
    +        sumAggregationStateType,
    +        sumAggregationRowType,
    +        false,
    +        queryConfig
    +        .withIdleStateRetentionTime(Time.seconds(4), Time.seconds(5))
    +        .withUnboundedAggregateUpdateInterval(Time.seconds(1))))
    +
    +    val testHarness =
    +      createHarnessTester(
    +        processFunction,
    +        new TupleRowKeySelector[String](2),
    +        BasicTypeInfo.STRING_TYPE_INFO)
    +
    +    testHarness.open()
    +
    +    testHarness.setProcessingTime(1)
    +
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
    +    testHarness.setProcessingTime(1000)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
    +
    +    testHarness.setProcessingTime(1002)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(4003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
    +
    +    // clear all states
    +    testHarness.setProcessingTime(10003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(12003)
    --- End diff --
    
    Add a case where a record is first added and later retracted (`CRow` flag is `false`) to verify that the operator sends out a delete message. Can be done on a new key (`"ccc"`).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125424099
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (currentTime >= triggerTimer) {
    +
    +      val newTimer = currentTime + queryConfig.getUnboundedAggregateUpdateInterval
    +
    +      emitTimerState.update(newTimer)
    +
    +      ctx.timerService().registerProcessingTimeTimer(newTimer)
    --- End diff --
    
    Yes, I think you are right, event-time timer is better. Because data driven is make sense in this PR. What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r123656803
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,215 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  private var emitRow: Row = _
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the last emit row
    +  private var preEmitState: ValueState[Row] = _
    +
    +  // stores the current emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +                s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    emitRow = function.createOutputRow
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    preEmitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", outputRowType))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (0 == triggerTimer || currentTime >= triggerTimer) {
    +
    +      // set group keys value to the final output
    +      function.setForwardedFields(input, emitRow)
    +      // set previous aggregate result to the prevRow
    +      function.setAggregationResults(accumulators, emitRow)
    --- End diff --
    
    Move `setAggregationResults` from `processElement` to `onTimer`. for 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4157: [FLINK-6649][table]Improve Non-window group aggregate wit...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4157
  
    I have update the PR. The change is Move setAggregationResults from processElement method to onTimer method. Benefits are:
    
    * Reduce the delay.
    * Reduce row copy.
    
    Welcome anybody to review.
    
    Best,
    SunJincheng


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125504087
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    --- End diff --
    
    Yes, that's true. :) 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125316569
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala ---
    @@ -80,18 +80,79 @@ class NonWindowHarnessTest extends HarnessTestBase {
     
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
    -
    -    verify(expectedOutput, result, new RowResultSortComparator(6))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, "bbb", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, "aaa", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, "aaa", 6: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, "aaa", 10: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, "bbb", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, "aaa", 5: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, "aaa", 11: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, "aaa", 18: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, "bbb", 3: JInt), true), 1))
    +
    +    verifySorted(expectedOutput, result, new RowResultSortComparator)
    +
    +    testHarness.close()
    +  }
    +
    +  @Test
    +  def testProcTimeNonWindowWithUpdateInterval(): Unit = {
    +
    +    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
    +      new GroupAggProcessFunctionWithUpdateInterval(
    +        genSumAggFunction,
    +        sumAggregationStateType,
    +        sumAggregationRowType,
    +        false,
    +        queryConfig
    +        .withIdleStateRetentionTime(Time.seconds(4), Time.seconds(5))
    +        .withUnboundedAggregateUpdateInterval(Time.seconds(1))))
    +
    +    val testHarness =
    +      createHarnessTester(
    +        processFunction,
    +        new TupleRowKeySelector[String](2),
    +        BasicTypeInfo.STRING_TYPE_INFO)
    +
    +    testHarness.open()
    +
    +    testHarness.setProcessingTime(1)
    +
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
    +    testHarness.setProcessingTime(1000)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
    +
    +    testHarness.setProcessingTime(1002)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(4003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
    +
    +    // clear all states
    +    testHarness.setProcessingTime(10003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(12003)
    --- End diff --
    
    Can you add this to the other tests as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125495801
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    --- End diff --
    
    I think if we drop the `ProcessFunction` part we should also rename `GroupAggProcessFunction` accordingly. Btw. since the same operator is also used for non-keyed aggregates, the `Group` part is also not really precise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125300423
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (currentTime >= triggerTimer) {
    +
    +      val newTimer = currentTime + queryConfig.getUnboundedAggregateUpdateInterval
    +
    +      emitTimerState.update(newTimer)
    +
    +      ctx.timerService().registerProcessingTimeTimer(newTimer)
    +    }
    +
    +  }
    +
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (needToCleanupState(timestamp)) {
    +      cleanupState(state, cntState, inputState, emitTimerState, emitState)
    +    } else {
    --- End diff --
    
    check that `timestamp == emitTimerState.value`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125504798
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    --- End diff --
    
    So, I think `NonWindowedAgg` and `IntervalUpdateNonWindowedAgg` is better. And I am fine if you want add `ProcessFunction` at the end. Because many class name end with xxxFunction.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125487643
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (currentTime >= triggerTimer) {
    +
    +      val newTimer = currentTime + queryConfig.getUnboundedAggregateUpdateInterval
    +
    +      emitTimerState.update(newTimer)
    +
    +      ctx.timerService().registerProcessingTimeTimer(newTimer)
    --- End diff --
    
    This is kind of tricky because non-windowed aggregations might also be defined on tables/streams without watermarks. 
    Maybe we need to introduce an additional parameter like for example `updateIntervalTimeMode` .


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125500774
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala ---
    @@ -37,6 +37,13 @@ class BatchQueryConfig private[table] extends QueryConfig
     class StreamQueryConfig private[table] extends QueryConfig {
     
       /**
    +    * The non-windowed groupby aggregate update the calculation result according a configuration of
    +    * time interval. By default non-windowed groupby aggregate will update calculation result each
    +    * row.
    +    */
    +  private var unboundedAggregateUpdateInterval: Time = Time.milliseconds(0)
    --- End diff --
    
    I think we need to achieve separately, i.e., proc-time non-windowed aggregations using `process-time timer.` and event-time non-windowed aggregations using event-time timer. `updateInterval` also make sense, because `updateInterval` is relative to the last emit time. i.e., emitTime = current(Proc/Row)Time+updateInterval. (Calculate emitTime when the data arrives).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125312324
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    --- End diff --
    
    Rename to `GroupAggUpdateRateProcessFunction` (or something else shorter)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125501792
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala ---
    @@ -80,18 +80,79 @@ class NonWindowHarnessTest extends HarnessTestBase {
     
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
    -
    -    verify(expectedOutput, result, new RowResultSortComparator(6))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, "bbb", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, "aaa", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, "aaa", 6: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, "aaa", 10: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, "bbb", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, "aaa", 5: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, "aaa", 11: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, "aaa", 18: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, "bbb", 3: JInt), true), 1))
    +
    +    verifySorted(expectedOutput, result, new RowResultSortComparator)
    +
    +    testHarness.close()
    +  }
    +
    +  @Test
    +  def testProcTimeNonWindowWithUpdateInterval(): Unit = {
    +
    +    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
    +      new GroupAggProcessFunctionWithUpdateInterval(
    +        genSumAggFunction,
    +        sumAggregationStateType,
    +        sumAggregationRowType,
    +        false,
    +        queryConfig
    +        .withIdleStateRetentionTime(Time.seconds(4), Time.seconds(5))
    +        .withUnboundedAggregateUpdateInterval(Time.seconds(1))))
    +
    +    val testHarness =
    +      createHarnessTester(
    +        processFunction,
    +        new TupleRowKeySelector[String](2),
    +        BasicTypeInfo.STRING_TYPE_INFO)
    +
    +    testHarness.open()
    +
    +    testHarness.setProcessingTime(1)
    +
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
    +    testHarness.setProcessingTime(1000)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
    +
    +    testHarness.setProcessingTime(1002)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(4003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
    +
    +    // clear all states
    +    testHarness.setProcessingTime(10003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(12003)
    +
    +    val result = testHarness.getOutput
    +
    +    val expectedOutput = new ConcurrentLinkedQueue[Object]()
    +
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 6: JInt), true), 1001))
    --- End diff --
    
    Yes, Agree with you. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125325560
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala ---
    @@ -80,18 +80,79 @@ class NonWindowHarnessTest extends HarnessTestBase {
     
         val expectedOutput = new ConcurrentLinkedQueue[Object]()
     
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
    -    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
    -
    -    verify(expectedOutput, result, new RowResultSortComparator(6))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, "bbb", 1: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, "aaa", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, "aaa", 6: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, "aaa", 10: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, "bbb", 3: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, "aaa", 5: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, "aaa", 11: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, "aaa", 18: JInt), true), 1))
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, "bbb", 3: JInt), true), 1))
    +
    +    verifySorted(expectedOutput, result, new RowResultSortComparator)
    +
    +    testHarness.close()
    +  }
    +
    +  @Test
    +  def testProcTimeNonWindowWithUpdateInterval(): Unit = {
    +
    +    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
    +      new GroupAggProcessFunctionWithUpdateInterval(
    +        genSumAggFunction,
    +        sumAggregationStateType,
    +        sumAggregationRowType,
    +        false,
    +        queryConfig
    +        .withIdleStateRetentionTime(Time.seconds(4), Time.seconds(5))
    +        .withUnboundedAggregateUpdateInterval(Time.seconds(1))))
    +
    +    val testHarness =
    +      createHarnessTester(
    +        processFunction,
    +        new TupleRowKeySelector[String](2),
    +        BasicTypeInfo.STRING_TYPE_INFO)
    +
    +    testHarness.open()
    +
    +    testHarness.setProcessingTime(1)
    +
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
    +    testHarness.setProcessingTime(1000)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
    +
    +    testHarness.setProcessingTime(1002)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(4003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
    +
    +    // clear all states
    +    testHarness.setProcessingTime(10003)
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
    +    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
    +
    +    testHarness.setProcessingTime(12003)
    +
    +    val result = testHarness.getOutput
    +
    +    val expectedOutput = new ConcurrentLinkedQueue[Object]()
    +
    +    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, "aaa", 6: JInt), true), 1001))
    --- End diff --
    
    This test is overly strict and tests behavior that is not required. In SQL (and the Table API) all fields must either be grouping keys or aggregation results. Here, the first attribute is neither a key or aggregated. Hence, the test validates that which rows are internally hold by checking a value that is according to SQL standards not defined. I'd modify the test such that it only checks grouping keys or aggregation results.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125498157
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    +      inputCnt = 0L
    +    }
    +
    +    // update aggregate result and set to the newRow
    +    if (inputC.change) {
    +      inputCnt += 1
    +      // accumulate input
    +      function.accumulate(accumulators, input)
    +    } else {
    +      inputCnt -= 1
    +      // retract input
    +      function.retract(accumulators, input)
    +    }
    +
    +    state.update(accumulators)
    +    cntState.update(inputCnt)
    +
    +    var triggerTimer = emitTimerState.value
    +
    +    if (null == triggerTimer) {
    +      triggerTimer = 0L
    +    }
    +
    +    if (currentTime >= triggerTimer) {
    +
    +      val newTimer = currentTime + queryConfig.getUnboundedAggregateUpdateInterval
    +
    +      emitTimerState.update(newTimer)
    +
    +      ctx.timerService().registerProcessingTimeTimer(newTimer)
    --- End diff --
    
    Yes, that's why I `registerProcessingTimeTimer` at the first-time. At current time, we only supported proc-time non-windowed aggregations. So, we can using `registerProcessingTimeTimer` for proc-time. When we supported event-time non-windowed aggregations we can using `registerEventTimeTimer`.  Because event-time non-windowed aggregations need an additional Implement. And I think we do not need `updateIntervalTimeMode` If we have different implementations for proc-time and event-time. 
    i.e., `updateInterval <= 0 ` is immediate update. `updateInterval > 0 ` is Interval update.
    What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4157: [FLINK-6649][table]Improve Non-window group aggregate wit...

Posted by sunjincheng121 <gi...@git.apache.org>.
Github user sunjincheng121 commented on the issue:

    https://github.com/apache/flink/pull/4157
  
    @wuchong @shaoxuan-wang @fhueske 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125495413
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala ---
    @@ -37,6 +37,13 @@ class BatchQueryConfig private[table] extends QueryConfig
     class StreamQueryConfig private[table] extends QueryConfig {
     
       /**
    +    * The non-windowed groupby aggregate update the calculation result according a configuration of
    +    * time interval. By default non-windowed groupby aggregate will update calculation result each
    +    * row.
    +    */
    +  private var unboundedAggregateUpdateInterval: Time = Time.milliseconds(0)
    --- End diff --
    
    I think the distinction makes sense but doubt that everybody would make the same distinction. I'm fine with `updateInterval` but I also think that we need an additional parameter to specify whether the update interval should be in event-time or processing-time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125304367
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    +  */
    +class GroupAggProcessFunctionWithUpdateInterval(
    +    private val genAggregations: GeneratedAggregationsFunction,
    +    private val aggregationStateType: RowTypeInfo,
    +    private val outputRowType: RowTypeInfo,
    +    private val generateRetraction: Boolean,
    +    private val queryConfig: StreamQueryConfig)
    +  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
    +with Compiler[GeneratedAggregations] {
    +
    +  protected val LOG: Logger = LoggerFactory.getLogger(this.getClass)
    +  protected var function: GeneratedAggregations = _
    +
    +  protected var newRow: CRow = _
    +  protected var prevRow: CRow = _
    +
    +  private var typeSerializer: TypeSerializer[Row] = _
    +
    +  // stores the accumulators
    +  protected var state: ValueState[Row] = _
    +
    +  // counts the number of added and retracted input records
    +  protected var cntState: ValueState[JLong] = _
    +
    +  // stores the input for group keys
    +  private var inputState: ValueState[Row] = _
    +
    +  // stores the last emit row
    +  private var emitState: ValueState[Row] = _
    +
    +  // stores the emit time
    +  private var emitTimerState: ValueState[JLong] = _
    +
    +
    +  override def open(config: Configuration) {
    +    LOG.debug(s"Compiling AggregateHelper: ${genAggregations.name}\n\n" +
    +      s"Code:\n${genAggregations.code}")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genAggregations.name,
    +      genAggregations.code)
    +    LOG.debug("Instantiating AggregateHelper.")
    +    function = clazz.newInstance()
    +
    +    newRow = new CRow(function.createOutputRow, true)
    +    prevRow = new CRow(function.createOutputRow, false)
    +    typeSerializer = outputRowType.createSerializer(new ExecutionConfig())
    +
    +    state = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType))
    +    cntState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG))
    +    inputState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregatePreEmitState", classOf[Row]))
    +    emitState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[Row]("GroupAggregateEmitState", outputRowType))
    +    emitTimerState = getRuntimeContext.getState(
    +      new ValueStateDescriptor[JLong]("emitTimeState", Types.LONG))
    +
    +    initCleanupTimeState("GroupAggregateWithUpdateIntervalCleanupTime")
    +  }
    +
    +  override def processElement(
    +      inputC: CRow,
    +      ctx: ProcessFunction[CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    val currentTime = ctx.timerService().currentProcessingTime()
    +    // register state-cleanup timer
    +    registerProcessingCleanupTimer(ctx, currentTime)
    +
    +    val input = inputC.row
    +
    +    // get accumulators and input counter
    +    var accumulators = state.value
    +    var inputCnt = cntState.value
    +
    +    if (null == accumulators) {
    +      accumulators = function.createAccumulators()
    +      inputState.update(input)
    --- End diff --
    
    I think we should store a row of type `outputRowType` in the `inputState` on which just the forwarded fields are set. 
    ```
    val latestRow = Row(outputArity)
    function.setForwardedFields(input, latestRow)
    inputState.update(latestRow)
    ```
    
    This should be 1) smaller than the raw input because aggregated fields are stripped of and 2) have the same type as `emitState` (`TypeInformation` is known). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4157: [FLINK-6649][table]Improve Non-window group aggreg...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4157#discussion_r125284189
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunctionWithUpdateInterval.scala ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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.{Long => JLong}
    +
    +import org.apache.flink.api.common.ExecutionConfig
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeutils.TypeSerializer
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.slf4j.{Logger, LoggerFactory}
    +
    +/**
    +  * Aggregate Function used for the groupby (without window) aggregate
    +  * with update interval config.
    +  *
    +  * @param genAggregations      Generated aggregate helper function
    +  * @param aggregationStateType The row type info of aggregation
    +  * @param outputRowType The row type info of output.
    --- End diff --
    
    Add remaining parameters as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---