Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-4557 Table API Stream Aggregations
  3. FLINK-5653

Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL

    Details

    • Type: Sub-task
    • Status: Closed
    • Priority: Major
    • Resolution: Implemented
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Table API & SQL
    • Labels:
      None

      Description

      The goal of this issue is to add support for OVER ROWS aggregations on processing time streams to the SQL interface.

      Queries similar to the following should be supported:

      SELECT 
        a, 
        SUM(b) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumB,
        MIN(b) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minB
      FROM myStream
      

      The following restrictions should initially apply:

      • All OVER clauses in the same SELECT clause must be exactly the same.
      • The PARTITION BY clause is optional (no partitioning results in single threaded execution).
      • The ORDER BY clause may only have procTime() as parameter. procTime() is a parameterless scalar function that just indicates processing time mode.
      • UNBOUNDED PRECEDING is not supported (see FLINK-5656)
      • FOLLOWING is not supported.

      The restrictions will be resolved in follow up issues. If we find that some of the restrictions are trivial to address, we can add the functionality in this issue as well.

      This issue includes:

      • Design of the DataStream operator to compute OVER ROW aggregates
      • Translation from Calcite's RelNode representation (LogicalProject with RexOver expression).

        Issue Links

          Activity

          Hide
          fhueske Fabian Hueske added a comment -

          Implemented with ee033c903b20d7a233009764b6b96e78eea5b981

          thanks for the contribution Stefano Bortoli!

          Show
          fhueske Fabian Hueske added a comment - Implemented with ee033c903b20d7a233009764b6b96e78eea5b981 thanks for the contribution Stefano Bortoli !
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3574
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3653
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Thanks for the update @stefanobortoli!
          Will merge this

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3653 Thanks for the update @stefanobortoli! Will merge this
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi, @stefanobortoli thanks for this PR. It's good for me. +1

          Cheers,
          SunJincheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3653 Hi, @stefanobortoli thanks for this PR. It's good for me. +1 Cheers, SunJincheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user stefanobortoli commented on the issue:

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

          @fhueske @sunjincheng121 the latest merge caused a conflict, however I have already pushed a new branch and created a new PR. This can be closed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske @sunjincheng121 the latest merge caused a conflict, however I have already pushed a new branch and created a new PR. This can be closed.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user stefanobortoli commented on the issue:

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

          @fhueske @sunjincheng121 I have merged with the most recent branch, using the function freshly merged. I think the PR is good to merge now.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on the issue: https://github.com/apache/flink/pull/3653 @fhueske @sunjincheng121 I have merged with the most recent branch, using the function freshly merged. I think the PR is good to merge now.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user stefanobortoli opened a pull request:

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

          FLINK-5653 Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL

          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-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [ X] 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/huawei-flink/flink FLINK-5653d

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

          https://github.com/apache/flink/pull/3653.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 #3653


          commit 5ca561b0a3cec68e9386286eb445275ba9b4ce77
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-03-30T09:28:41Z

          Over aggregation with row range ang procTime semantic


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user stefanobortoli opened a pull request: https://github.com/apache/flink/pull/3653 FLINK-5653 Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL 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-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [ X] 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/huawei-flink/flink FLINK-5653 d Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3653.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 #3653 commit 5ca561b0a3cec68e9386286eb445275ba9b4ce77 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-03-30T09:28:41Z Over aggregation with row range ang procTime semantic
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @stefanobortoli, I wanted to merge your PR but found that there is still a merge commit included.
          Could you please remove that commit and update the PR branch (force push).

          Thanks, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @stefanobortoli, I wanted to merge your PR but found that there is still a merge commit included. Could you please remove that commit and update the PR branch (force push). Thanks, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Thanks for the quick update @stefanobortoli
          No worries about the extra commit. I'll squash those before merging.

          Thanks, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Thanks for the quick update @stefanobortoli No worries about the extra commit. I'll squash those before merging. Thanks, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user stefanobortoli commented on the issue:

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

          @fhueske @sunjincheng121 @rtudoran sorry for the extra commit, I saw some comments too little too late. Now the code should comply with the requested changes.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske @sunjincheng121 @rtudoran sorry for the extra commit, I saw some comments too little too late. Now the code should comply with the requested changes.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108703530

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          — End diff –

          yes, exactly

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108703530 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 — End diff – yes, exactly
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108703268

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          — End diff –

          So, you suggest to update the counterState only when the value is +1 and when it reaches a steady state, it should not be updated. Ok.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108703268 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 — End diff – So, you suggest to update the counterState only when the value is +1 and when it reaches a steady state, it should not be updated. Ok.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108693863

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          + // if reference timestamp list not empty, keep the list
          + if (!retractList.isEmpty())

          { + rowMapState.put(retractTs, retractList) + }

          // if smallest timestamp list is empty, remove and find new smallest
          + else {
          + rowMapState.remove(retractTs)
          + val iter = rowMapState.keys.iterator()
          + var currentTs: Long = 0L
          + var newSmallesTs: Long = Long.MaxValue
          + while(iter.hasNext){
          + currentTs = iter.next
          + if(currentTs < newSmallesTs)

          { + newSmallesTs = currentTs + }

          + }
          + smallestTs = newSmallesTs
          + }
          + }
          +
          + // copy forwarded fields in output row
          + i = 0
          + while (i < forwardedFieldCount)

          { + output.setField(i, input.getField(i)) + i += 1 + }

          +
          + // accumulate current row and set aggregate in output row
          + i = 0
          + while (i < aggregates.length)

          { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, input.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + }

          +
          + // update map state, accumulator state, counter and timestamp
          + if (rowMapState.contains(currentTime))

          { + rowMapState.get(currentTime).add(input) + }

          else

          { // add new input + val newList = new ArrayList[Row] + newList.add(input) + rowMapState.put(currentTime, newList) + }

          + counter += 1
          + accumulatorState.update(accumulators)
          + smallestTsState.update(smallestTs)
          — End diff –

          we only need to update the `smallestTsState` and `counterState` if we actually changed the value (see comments above).

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108693863 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 + // if reference timestamp list not empty, keep the list + if (!retractList.isEmpty()) { + rowMapState.put(retractTs, retractList) + } // if smallest timestamp list is empty, remove and find new smallest + else { + rowMapState.remove(retractTs) + val iter = rowMapState.keys.iterator() + var currentTs: Long = 0L + var newSmallesTs: Long = Long.MaxValue + while(iter.hasNext){ + currentTs = iter.next + if(currentTs < newSmallesTs) { + newSmallesTs = currentTs + } + } + smallestTs = newSmallesTs + } + } + + // copy forwarded fields in output row + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, input.getField(i)) + i += 1 + } + + // accumulate current row and set aggregate in output row + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, input.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + + // update map state, accumulator state, counter and timestamp + if (rowMapState.contains(currentTime)) { + rowMapState.get(currentTime).add(input) + } else { // add new input + val newList = new ArrayList[Row] + newList.add(input) + rowMapState.put(currentTime, newList) + } + counter += 1 + accumulatorState.update(accumulators) + smallestTsState.update(smallestTs) — End diff – we only need to update the `smallestTsState` and `counterState` if we actually changed the value (see comments above).
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108693192

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          + // if reference timestamp list not empty, keep the list
          + if (!retractList.isEmpty())

          { + rowMapState.put(retractTs, retractList) + }

          // if smallest timestamp list is empty, remove and find new smallest
          + else {
          + rowMapState.remove(retractTs)
          + val iter = rowMapState.keys.iterator()
          + var currentTs: Long = 0L
          + var newSmallesTs: Long = Long.MaxValue
          + while(iter.hasNext){
          + currentTs = iter.next
          + if(currentTs < newSmallesTs)

          { + newSmallesTs = currentTs + }

          + }
          + smallestTs = newSmallesTs
          + }
          + }
          +
          + // copy forwarded fields in output row
          + i = 0
          + while (i < forwardedFieldCount)

          { + output.setField(i, input.getField(i)) + i += 1 + }

          +
          + // accumulate current row and set aggregate in output row
          + i = 0
          + while (i < aggregates.length)

          { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, input.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + }

          +
          + // update map state, accumulator state, counter and timestamp
          + if (rowMapState.contains(currentTime)) {
          + rowMapState.get(currentTime).add(input)
          — End diff –

          This will not modify the state but just the list object returned from the state. We need to explicitly `put` the updated list into the `MapState`. It works for in-memory state backends because all objects are hold on the heap.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108693192 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 + // if reference timestamp list not empty, keep the list + if (!retractList.isEmpty()) { + rowMapState.put(retractTs, retractList) + } // if smallest timestamp list is empty, remove and find new smallest + else { + rowMapState.remove(retractTs) + val iter = rowMapState.keys.iterator() + var currentTs: Long = 0L + var newSmallesTs: Long = Long.MaxValue + while(iter.hasNext){ + currentTs = iter.next + if(currentTs < newSmallesTs) { + newSmallesTs = currentTs + } + } + smallestTs = newSmallesTs + } + } + + // copy forwarded fields in output row + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, input.getField(i)) + i += 1 + } + + // accumulate current row and set aggregate in output row + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, input.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + + // update map state, accumulator state, counter and timestamp + if (rowMapState.contains(currentTime)) { + rowMapState.get(currentTime).add(input) — End diff – This will not modify the state but just the list object returned from the state. We need to explicitly `put` the updated list into the `MapState`. It works for in-memory state backends because all objects are hold on the heap.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108690587

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          — End diff –

          is overridden every time `processElement` is called. Does not need to be a member variable.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108690587 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ — End diff – is overridden every time `processElement` is called. Does not need to be a member variable.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108687937

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration) {
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + // We keep the elements received in a list state
          + // together with the ingestion time in the operator
          + // we also keep counter of processed elements
          + // and timestamp of oldest element
          + val rowListTypeInfo: TypeInformation[JList[Row]] =
          + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]]
          +
          + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
          + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState",
          + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo)
          +
          + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor)
          +
          + val stateDescriptor: ValueStateDescriptor[Row] =
          + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo)
          +
          + accumulatorState = getRuntimeContext.getState(stateDescriptor)
          +
          + val processedCountDescriptor : ValueStateDescriptor[Long] =
          + new ValueStateDescriptor[Long]("processedCountState", classOf[Long])
          +
          + counterState = getRuntimeContext.getState(processedCountDescriptor)
          +
          + val smallesTimestampDescriptor : ValueStateDescriptor[Long] =
          + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long])
          — End diff –

          +t -> "smallestTSState"

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108687937 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList [Row] ] = + new ListTypeInfo [Row] (inputType).asInstanceOf[TypeInformation[JList [Row] ]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList [Row] ] = + new MapStateDescriptor[Long, JList [Row] ]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation [Long] ], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor [Row] = + new ValueStateDescriptor [Row] ("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor [Long] = + new ValueStateDescriptor [Long] ("processedCountState", classOf [Long] ) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor [Long] = + new ValueStateDescriptor [Long] ("smallesTSState", classOf [Long] ) — End diff – +t -> "smallestTSState"
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108686536

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          — End diff –

          does not need to be a member of the function and can be defined in `processElement`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108686536 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ — End diff – does not need to be a member of the function and can be defined in `processElement`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108694667

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          + // if reference timestamp list not empty, keep the list
          + if (!retractList.isEmpty())

          { + rowMapState.put(retractTs, retractList) + }

          // if smallest timestamp list is empty, remove and find new smallest
          + else {
          + rowMapState.remove(retractTs)
          + val iter = rowMapState.keys.iterator()
          + var currentTs: Long = 0L
          + var newSmallesTs: Long = Long.MaxValue
          — End diff –

          +t -> `newSmallestTs`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108694667 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 + // if reference timestamp list not empty, keep the list + if (!retractList.isEmpty()) { + rowMapState.put(retractTs, retractList) + } // if smallest timestamp list is empty, remove and find new smallest + else { + rowMapState.remove(retractTs) + val iter = rowMapState.keys.iterator() + var currentTs: Long = 0L + var newSmallesTs: Long = Long.MaxValue — End diff – +t -> `newSmallestTs`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108686114

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          — End diff –

          `LinkedList` import is not used

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108686114 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } — End diff – `LinkedList` import is not used
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108691088

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          — End diff –

          Once `counter` is equal to `precedingOffset` we always have to accumulate and retract. So we only have to increment it as long as it is smaller. This can be done in an `else` branch of the retraction condition (`counter == precedingOffset`). Also we only need to update the `counterState` if we increment the counter.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108691088 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 — End diff – Once `counter` is equal to `precedingOffset` we always have to accumulate and retract. So we only have to increment it as long as it is smaller. This can be done in an `else` branch of the retraction condition (`counter == precedingOffset`). Also we only need to update the `counterState` if we increment the counter.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108689351

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          — End diff –

          is overridden every time `processElement` is called. Does not need to be a member variable.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108689351 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ — End diff – is overridden every time `processElement` is called. Does not need to be a member variable.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108695786

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          + // if reference timestamp list not empty, keep the list
          + if (!retractList.isEmpty()) {
          — End diff –

          please remove `()` from "get-style" method without parameters.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108695786 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 + // if reference timestamp list not empty, keep the list + if (!retractList.isEmpty()) { — End diff – please remove `()` from "get-style" method without parameters.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108685193

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -730,6 +736,41 @@ object AggregateUtil

          { (aggFunction, accumulatorRowType, aggResultRowType) }

          + /**
          + * Function for building the processing logic for aggregating data in row bounded windows
          + *
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @param rowType Type info of row
          + * @param precedingOffset The window lower boundary expressed in number of rows
          + * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
          + */
          + private[flink] def createBoundedProcessingOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType,
          + rowType: RowTypeInfo,
          + precedingOffset: Int): ProcessFunction[Row, Row] = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = true)
          +
          + val aggregationStateType: RowTypeInfo =
          + createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
          — End diff –

          can be replaced by `createAccumulatorRowType(aggregates)`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108685193 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -730,6 +736,41 @@ object AggregateUtil { (aggFunction, accumulatorRowType, aggResultRowType) } + /** + * Function for building the processing logic for aggregating data in row bounded windows + * + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @param rowType Type info of row + * @param precedingOffset The window lower boundary expressed in number of rows + * @return [ [org.apache.flink.streaming.api.functions.ProcessFunction] ] + */ + private [flink] def createBoundedProcessingOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType, + rowType: RowTypeInfo, + precedingOffset: Int): ProcessFunction [Row, Row] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = true) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) — End diff – can be replaced by `createAccumulatorRowType(aggregates)`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108693315

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          + // if reference timestamp list not empty, keep the list
          + if (!retractList.isEmpty())

          { + rowMapState.put(retractTs, retractList) + }

          // if smallest timestamp list is empty, remove and find new smallest
          + else {
          + rowMapState.remove(retractTs)
          + val iter = rowMapState.keys.iterator()
          + var currentTs: Long = 0L
          + var newSmallesTs: Long = Long.MaxValue
          + while(iter.hasNext){
          + currentTs = iter.next
          + if(currentTs < newSmallesTs)

          { + newSmallesTs = currentTs + }

          + }
          + smallestTs = newSmallesTs
          + }
          + }
          +
          + // copy forwarded fields in output row
          + i = 0
          + while (i < forwardedFieldCount)

          { + output.setField(i, input.getField(i)) + i += 1 + }

          +
          + // accumulate current row and set aggregate in output row
          + i = 0
          + while (i < aggregates.length)

          { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, input.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + }

          +
          + // update map state, accumulator state, counter and timestamp
          + if (rowMapState.contains(currentTime)) {
          — End diff –

          We can safe a state access if we do not call `contains` but `get` and check for `null`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108693315 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 + // if reference timestamp list not empty, keep the list + if (!retractList.isEmpty()) { + rowMapState.put(retractTs, retractList) + } // if smallest timestamp list is empty, remove and find new smallest + else { + rowMapState.remove(retractTs) + val iter = rowMapState.keys.iterator() + var currentTs: Long = 0L + var newSmallesTs: Long = Long.MaxValue + while(iter.hasNext){ + currentTs = iter.next + if(currentTs < newSmallesTs) { + newSmallesTs = currentTs + } + } + smallestTs = newSmallesTs + } + } + + // copy forwarded fields in output row + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, input.getField(i)) + i += 1 + } + + // accumulate current row and set aggregate in output row + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, input.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + + // update map state, accumulator state, counter and timestamp + if (rowMapState.contains(currentTime)) { — End diff – We can safe a state access if we do not call `contains` but `get` and check for `null`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108686885

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration) {
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + // We keep the elements received in a list state
          — End diff –

          Update comment. We use a `MapState` keyed by the ingestion time.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108686885 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state — End diff – Update comment. We use a `MapState` keyed by the ingestion time.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108692025

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          + val retractList = rowMapState.get(smallestTs)
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + }

          + retractList.remove(0)
          + counter -= 1
          + // if reference timestamp list not empty, keep the list
          + if (!retractList.isEmpty())

          { + rowMapState.put(retractTs, retractList) + }

          // if smallest timestamp list is empty, remove and find new smallest
          + else {
          + rowMapState.remove(retractTs)
          + val iter = rowMapState.keys.iterator()
          + var currentTs: Long = 0L
          + var newSmallesTs: Long = Long.MaxValue
          + while(iter.hasNext){
          + currentTs = iter.next
          + if(currentTs < newSmallesTs)

          { + newSmallesTs = currentTs + }

          + }
          + smallestTs = newSmallesTs
          — End diff –

          We can immediately update the `smallestTsState` here and avoid updating if if the timestamp was not changed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108692025 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs + val retractList = rowMapState.get(smallestTs) + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i))) + i += 1 + } + retractList.remove(0) + counter -= 1 + // if reference timestamp list not empty, keep the list + if (!retractList.isEmpty()) { + rowMapState.put(retractTs, retractList) + } // if smallest timestamp list is empty, remove and find new smallest + else { + rowMapState.remove(retractTs) + val iter = rowMapState.keys.iterator() + var currentTs: Long = 0L + var newSmallesTs: Long = Long.MaxValue + while(iter.hasNext){ + currentTs = iter.next + if(currentTs < newSmallesTs) { + newSmallesTs = currentTs + } + } + smallestTs = newSmallesTs — End diff – We can immediately update the `smallestTsState` here and avoid updating if if the timestamp was not changed.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108694404

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase

          { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + //////////////////////////////////////////////////////
          + // START TESTING BOUNDED PROC TIME ROW AGGREGATION
          + //////////////////////////////////////////////////////
          +
          +
          + @Test
          + def testSumMinPartitionedAggregatation2(): Unit = {
          +
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          — End diff –

          please add `env.setStateBackend(getStateBackend)` to all tests.
          This will use the RocksDBStateBackend for the tests.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108694404 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + ////////////////////////////////////////////////////// + // START TESTING BOUNDED PROC TIME ROW AGGREGATION + ////////////////////////////////////////////////////// + + + @Test + def testSumMinPartitionedAggregatation2(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment — End diff – please add `env.setStateBackend(getStateBackend)` to all tests. This will use the RocksDBStateBackend for the tests.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108692279

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,182 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val precedingOffset: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(precedingOffset > 0)
          +
          + private var accumulators: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var output: Row = _
          + private var counterState: ValueState[Long] = _
          + private var counter : Long = _
          + private var smallestTsState: ValueState[Long] = _
          + private var smallestTs : Long = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + // get smallest timestamp
          + smallestTs = smallestTsState.value()
          + if(smallestTs == 0L)

          { + smallestTs = currentTime + }

          + // get previous counter value
          + counter = counterState.value()
          +
          + if (counter == precedingOffset) {
          + val retractTs = smallestTs
          — End diff –

          I think we do not need `retractTs`. It is always equal to `smallestTs`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108692279 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,182 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val precedingOffset: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(precedingOffset > 0) + + private var accumulators: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var output: Row = _ + private var counterState: ValueState [Long] = _ + private var counter : Long = _ + private var smallestTsState: ValueState [Long] = _ + private var smallestTs : Long = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + // we also keep counter of processed elements + // and timestamp of oldest element + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + + val processedCountDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("processedCountState", classOf[Long]) + + counterState = getRuntimeContext.getState(processedCountDescriptor) + + val smallesTimestampDescriptor : ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("smallesTSState", classOf[Long]) + + smallestTsState = getRuntimeContext.getState(smallesTimestampDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // get smallest timestamp + smallestTs = smallestTsState.value() + if(smallestTs == 0L) { + smallestTs = currentTime + } + // get previous counter value + counter = counterState.value() + + if (counter == precedingOffset) { + val retractTs = smallestTs — End diff – I think we do not need `retractTs`. It is always equal to `smallestTs`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108644799

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase

          { streamUtil.verifySql(sql, expected) }

          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          — End diff –

          It's Okay.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108644799 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + — End diff – It's Okay.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108626504

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase

          { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + //////////////////////////////////////////////////////
          + // START TESTING BOUNDED PROC TIME ROW AGGREGATION
          + //////////////////////////////////////////////////////
          +
          +
          + @Test
          + def testSumMinAggregatation2(): Unit = {
          — End diff –

          If you had checked with @fhueske I'm okay. IMO. the test method name is not friendly.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108626504 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + ////////////////////////////////////////////////////// + // START TESTING BOUNDED PROC TIME ROW AGGREGATION + ////////////////////////////////////////////////////// + + + @Test + def testSumMinAggregatation2(): Unit = { — End diff – If you had checked with @fhueske I'm okay. IMO. the test method name is not friendly.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108626444

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase

          { streamUtil.verifySql(sql, expected) }

          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          — End diff –

          I have tested 4 situations that Fabian thought to be sufficient for our purpose. 3 or 4 does not change besides my work to manually assemble the test. of course unless the aggregations are not reliable and summing 3 numbers or 4 numbers could have an impact. :-D

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108626444 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + — End diff – I have tested 4 situations that Fabian thought to be sufficient for our purpose. 3 or 4 does not change besides my work to manually assemble the test. of course unless the aggregations are not reliable and summing 3 numbers or 4 numbers could have an impact. :-D
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108625887

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          — End diff –

          More than the elements processed (which should be kept in the aggregation) I keep the number of elements in the buffer

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108625887 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 — End diff – More than the elements processed (which should be kept in the aggregation) I keep the number of elements in the buffer
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108611842

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          + while(keyIter.hasNext){
          + currentKeyTime = keyIter.next
          + i += rowMapState.get(currentKeyTime).size()
          + if(currentKeyTime <= oldestTimeStamp){
          + oldestTimeStamp = currentKeyTime
          + toRetract = rowMapState.get(currentKeyTime)
          — End diff –

          We only need move `toRetract = rowMapState.get(currentKeyTime)` to loop out, then `toRetract = rowMapState.get(oldestTimeStamp)`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108611842 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 + while(keyIter.hasNext){ + currentKeyTime = keyIter.next + i += rowMapState.get(currentKeyTime).size() + if(currentKeyTime <= oldestTimeStamp){ + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) — End diff – We only need move `toRetract = rowMapState.get(currentKeyTime)` to loop out, then `toRetract = rowMapState.get(oldestTimeStamp)`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108611370

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase

          { streamUtil.verifySql(sql, expected) }

          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          — End diff –

          Just a suggestion. the two test use different preceding offset that can test more situations.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108611370 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + — End diff – Just a suggestion. the two test use different preceding offset that can test more situations.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108611315

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -195,6 +194,45 @@ class DataStreamOverAggregate(
          result
          }

          + def createBoundedAndCurrentRowProcessingTimeOverWindow(
          + inputDS: DataStream[Row]): DataStream[Row] = {
          +
          + val overWindow: Group = logicWindow.groups.get(0)
          + val partitionKeys: Array[Int] = overWindow.keys.toArray
          + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + // window size is lowerbound +1 to comply with over semantics
          + val lowerbound: Int = AggregateUtil.getLowerBoundary(
          — End diff –

          sure

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108611315 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -195,6 +194,45 @@ class DataStreamOverAggregate( result } + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( — End diff – sure
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108611259

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase

          { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + //////////////////////////////////////////////////////
          + // START TESTING BOUNDED PROC TIME ROW AGGREGATION
          + //////////////////////////////////////////////////////
          +
          +
          + @Test
          + def testSumMinAggregatation2(): Unit = {
          — End diff –

          @fhueske suggested the 4 test, and 4 I implemented. It is the 5th time I implement the tests, we can leave it like that.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108611259 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + ////////////////////////////////////////////////////// + // START TESTING BOUNDED PROC TIME ROW AGGREGATION + ////////////////////////////////////////////////////// + + + @Test + def testSumMinAggregatation2(): Unit = { — End diff – @fhueske suggested the 4 test, and 4 I implemented. It is the 5th time I implement the tests, we can leave it like that.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108610311

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase

          { streamUtil.verifySql(sql, expected) }

          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          — End diff –

          why?

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108610311 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + — End diff – why?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108610149

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          + while(keyIter.hasNext){
          + currentKeyTime = keyIter.next
          + i += rowMapState.get(currentKeyTime).size()
          + if(currentKeyTime <= oldestTimeStamp){
          + oldestTimeStamp = currentKeyTime
          + toRetract = rowMapState.get(currentKeyTime)
          — End diff –

          you mean another state variable?

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108610149 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 + while(keyIter.hasNext){ + currentKeyTime = keyIter.next + i += rowMapState.get(currentKeyTime).size() + if(currentKeyTime <= oldestTimeStamp){ + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) — End diff – you mean another state variable?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108610034

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          — End diff –

          details, but ok

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108610034 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null — End diff – details, but ok
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108609894

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          — End diff –

          Very good suggestion, I was indeed wandering whether there was a better way to do that.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108609894 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 — End diff – Very good suggestion, I was indeed wandering whether there was a better way to do that.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108595908

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase

          { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + //////////////////////////////////////////////////////
          + // START TESTING BOUNDED PROC TIME ROW AGGREGATION
          + //////////////////////////////////////////////////////
          +
          +
          + @Test
          + def testSumMinAggregatation2(): Unit = {
          — End diff –

          I think we have two tests is enough, one is` partitioned` anther is `non-partitioned`. What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108595908 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -696,6 +696,181 @@ class SqlITCase extends StreamingWithStateTestBase { "6,8,Hello world,51,9,5,9,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + ////////////////////////////////////////////////////// + // START TESTING BOUNDED PROC TIME ROW AGGREGATION + ////////////////////////////////////////////////////// + + + @Test + def testSumMinAggregatation2(): Unit = { — End diff – I think we have two tests is enough, one is` partitioned` anther is `non-partitioned`. What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108600245

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase

          { streamUtil.verifySql(sql, expected) }

          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          — End diff –

          Change the number of `preceding` to greater than 3, for example: `4 preceding`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108600245 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + — End diff – Change the number of `preceding` to greater than 3, for example: `4 preceding`.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108596582

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -729,6 +747,39 @@ object AggregateUtil

          { (aggFunction, accumulatorRowType, aggResultRowType) }

          +
          +
          + /*
          + * Function for building the processing logic for aggregating data in row bounded windows
          — End diff –

          Please format it as follow:

          ```
          /**

          • Function for building the processing logic for aggregating data in row bounded windows
            *
          • @param namedAggregates List of calls to aggregate functions and their output field names
          • @param inputType Input row type
          • @param rowType Type info of row
          • @param lowerBound the window boundary
          • @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
            */
            ```
          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108596582 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -729,6 +747,39 @@ object AggregateUtil { (aggFunction, accumulatorRowType, aggResultRowType) } + + + /* + * Function for building the processing logic for aggregating data in row bounded windows — End diff – Please format it as follow: ``` /** Function for building the processing logic for aggregating data in row bounded windows * @param namedAggregates List of calls to aggregate functions and their output field names @param inputType Input row type @param rowType Type info of row @param lowerBound the window boundary @return [ [org.apache.flink.streaming.api.functions.ProcessFunction] ] */ ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108598458

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          + while(keyIter.hasNext){
          + currentKeyTime = keyIter.next
          + i += rowMapState.get(currentKeyTime).size()
          + if(currentKeyTime <= oldestTimeStamp)

          { + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) + }

          + }
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + if(i == bufferSize){
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, toRetract.get(0).getField(aggFields(i))) + i += 1 + }

          + toRetract.remove(0)
          + if(!toRetract.isEmpty())

          { + rowMapState.put(oldestTimeStamp, toRetract) + }

          else

          { + rowMapState.remove(oldestTimeStamp) + }

          + }
          +
          + //carry on the last element data with aggregates
          — End diff –

          // copy forwarded fields to output row

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108598458 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 + while(keyIter.hasNext){ + currentKeyTime = keyIter.next + i += rowMapState.get(currentKeyTime).size() + if(currentKeyTime <= oldestTimeStamp) { + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) + } + } + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + if(i == bufferSize){ + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, toRetract.get(0).getField(aggFields(i))) + i += 1 + } + toRetract.remove(0) + if(!toRetract.isEmpty()) { + rowMapState.put(oldestTimeStamp, toRetract) + } else { + rowMapState.remove(oldestTimeStamp) + } + } + + //carry on the last element data with aggregates — End diff – // copy forwarded fields to output row
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108596940

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -729,6 +747,39 @@ object AggregateUtil

          { (aggFunction, accumulatorRowType, aggResultRowType) }

          +
          +
          + /*
          + * Function for building the processing logic for aggregating data in row bounded windows
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @param rowType Type info of row
          + * @param lowerBound the window boundary
          + * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
          + */
          + private[flink] def createBoundedProcessingOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType,
          + rowType: RowTypeInfo,
          + lowerBound: Int): ProcessFunction[Row, Row] = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = true)
          +
          + val aggregationStateType: RowTypeInfo =
          + createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
          +
          + new BoundedProcessingOverRowProcessFunction(
          + aggregates,
          + aggFields, lowerBound,
          + inputType.getFieldCount,
          + aggregationStateType, FlinkTypeFactory.toInternalRowTypeInfo(inputType))
          — End diff –

          Use the new line.
          `aggregationStateType, FlinkTypeFactory.toInternalRowTypeInfo(inputType) `->

          ```
          aggregationStateType,
          FlinkTypeFactory.toInternalRowTypeInfo(inputType)
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108596940 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -729,6 +747,39 @@ object AggregateUtil { (aggFunction, accumulatorRowType, aggResultRowType) } + + + /* + * Function for building the processing logic for aggregating data in row bounded windows + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @param rowType Type info of row + * @param lowerBound the window boundary + * @return [ [org.apache.flink.streaming.api.functions.ProcessFunction] ] + */ + private [flink] def createBoundedProcessingOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType, + rowType: RowTypeInfo, + lowerBound: Int): ProcessFunction [Row, Row] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = true) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverRowProcessFunction( + aggregates, + aggFields, lowerBound, + inputType.getFieldCount, + aggregationStateType, FlinkTypeFactory.toInternalRowTypeInfo(inputType)) — End diff – Use the new line. `aggregationStateType, FlinkTypeFactory.toInternalRowTypeInfo(inputType) `-> ``` aggregationStateType, FlinkTypeFactory.toInternalRowTypeInfo(inputType) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108598502

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          + while(keyIter.hasNext){
          + currentKeyTime = keyIter.next
          + i += rowMapState.get(currentKeyTime).size()
          + if(currentKeyTime <= oldestTimeStamp)

          { + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) + }

          + }
          +
          + // get oldest element beyond buffer size
          + // and if oldest element exist, retract value
          + if(i == bufferSize){
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, toRetract.get(0).getField(aggFields(i))) + i += 1 + }

          + toRetract.remove(0)
          + if(!toRetract.isEmpty())

          { + rowMapState.put(oldestTimeStamp, toRetract) + }

          else

          { + rowMapState.remove(oldestTimeStamp) + }

          + }
          +
          + //carry on the last element data with aggregates
          + i = 0
          + while (i < forwardedFieldCount)

          { + output.setField(i, input.getField(i)) + i += 1 + }

          +
          + //accumulate last value
          — End diff –

          // accumulate current row and set aggregate in output row

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108598502 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 + while(keyIter.hasNext){ + currentKeyTime = keyIter.next + i += rowMapState.get(currentKeyTime).size() + if(currentKeyTime <= oldestTimeStamp) { + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) + } + } + + // get oldest element beyond buffer size + // and if oldest element exist, retract value + if(i == bufferSize){ + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).retract(accumulator, toRetract.get(0).getField(aggFields(i))) + i += 1 + } + toRetract.remove(0) + if(!toRetract.isEmpty()) { + rowMapState.put(oldestTimeStamp, toRetract) + } else { + rowMapState.remove(oldestTimeStamp) + } + } + + //carry on the last element data with aggregates + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, input.getField(i)) + i += 1 + } + + //accumulate last value — End diff – // accumulate current row and set aggregate in output row
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108599930

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          — End diff –

          I think we can use an `dataCountState` ValueState to keep the number of elements processed which is useful for failover. What do you think ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108599930 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 — End diff – I think we can use an `dataCountState` ValueState to keep the number of elements processed which is useful for failover. What do you think ?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108597142

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          — End diff –

          remove `var` declaration.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108597142 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() — End diff – remove `var` declaration.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108599214

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          + var currentKeyTime: Long = 0L
          + i = 0
          + while(keyIter.hasNext){
          + currentKeyTime = keyIter.next
          + i += rowMapState.get(currentKeyTime).size()
          + if(currentKeyTime <= oldestTimeStamp){
          + oldestTimeStamp = currentKeyTime
          + toRetract = rowMapState.get(currentKeyTime)
          — End diff –

          we can keep the smallest timestamp, then this can be moved on loop out.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108599214 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null + var currentKeyTime: Long = 0L + i = 0 + while(keyIter.hasNext){ + currentKeyTime = keyIter.next + i += rowMapState.get(currentKeyTime).size() + if(currentKeyTime <= oldestTimeStamp){ + oldestTimeStamp = currentKeyTime + toRetract = rowMapState.get(currentKeyTime) — End diff – we can keep the smallest timestamp, then this can be moved on loop out.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108597029

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          — End diff –

          remove no use import.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108597029 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint — End diff – remove no use import.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108596217

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -195,6 +194,45 @@ class DataStreamOverAggregate(
          result
          }

          + def createBoundedAndCurrentRowProcessingTimeOverWindow(
          + inputDS: DataStream[Row]): DataStream[Row] = {
          +
          + val overWindow: Group = logicWindow.groups.get(0)
          + val partitionKeys: Array[Int] = overWindow.keys.toArray
          + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + // window size is lowerbound +1 to comply with over semantics
          + val lowerbound: Int = AggregateUtil.getLowerBoundary(
          — End diff –

          Can you use `OverAggregate#getLowerBoundary` ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108596217 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -195,6 +194,45 @@ class DataStreamOverAggregate( result } + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( — End diff – Can you use `OverAggregate#getLowerBoundary` ?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108599456

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -129,7 +129,7 @@ class SqlITCase extends StreamingWithStateTestBase {
          val result = tEnv.sql(sqlQuery).toDataStream[Row]
          result.addSink(new StreamITCase.StringSink)
          env.execute()
          -
          +
          — End diff –

          Restore this line

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108599456 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -129,7 +129,7 @@ class SqlITCase extends StreamingWithStateTestBase { val result = tEnv.sql(sqlQuery).toDataStream [Row] result.addSink(new StreamITCase.StringSink) env.execute() - + — End diff – Restore this line
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108596468

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +95,18 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + def getLowerBoundary(
          — End diff –

          remove this method.Because `OverAggregate#getLowerBoundary` already exists.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108596468 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +95,18 @@ object AggregateUtil { aggregationStateType) } } + + def getLowerBoundary( — End diff – remove this method.Because `OverAggregate#getLowerBoundary` already exists.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108598345

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          — End diff –

          I suggest use `precedingOffset`, What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108598345 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, — End diff – I suggest use `precedingOffset`, What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108596688

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -729,6 +747,39 @@ object AggregateUtil

          { (aggFunction, accumulatorRowType, aggResultRowType) }

          +
          +
          + /*
          + * Function for building the processing logic for aggregating data in row bounded windows
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @param rowType Type info of row
          + * @param lowerBound the window boundary
          + * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
          + */
          + private[flink] def createBoundedProcessingOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType,
          + rowType: RowTypeInfo,
          + lowerBound: Int): ProcessFunction[Row, Row] = {
          — End diff –

          I suggest use `precedingOffset`, What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108596688 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -729,6 +747,39 @@ object AggregateUtil { (aggFunction, accumulatorRowType, aggResultRowType) } + + + /* + * Function for building the processing logic for aggregating data in row bounded windows + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @param rowType Type info of row + * @param lowerBound the window boundary + * @return [ [org.apache.flink.streaming.api.functions.ProcessFunction] ] + */ + private [flink] def createBoundedProcessingOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType, + rowType: RowTypeInfo, + lowerBound: Int): ProcessFunction [Row, Row] = { — End diff – I suggest use `precedingOffset`, What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108598268

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -729,6 +747,39 @@ object AggregateUtil

          { (aggFunction, accumulatorRowType, aggResultRowType) }

          +
          +
          + /*
          + * Function for building the processing logic for aggregating data in row bounded windows
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @param rowType Type info of row
          + * @param lowerBound the window boundary
          + * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
          + */
          + private[flink] def createBoundedProcessingOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType,
          + rowType: RowTypeInfo,
          + lowerBound: Int): ProcessFunction[Row, Row] = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = true)
          +
          + val aggregationStateType: RowTypeInfo =
          + createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
          +
          + new BoundedProcessingOverRowProcessFunction(
          + aggregates,
          + aggFields, lowerBound,
          — End diff –

          `aggFields, lowerBound,` ->
          ` aggFields,`
          `lowerBound,`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108598268 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -729,6 +747,39 @@ object AggregateUtil { (aggFunction, accumulatorRowType, aggResultRowType) } + + + /* + * Function for building the processing logic for aggregating data in row bounded windows + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @param rowType Type info of row + * @param lowerBound the window boundary + * @return [ [org.apache.flink.streaming.api.functions.ProcessFunction] ] + */ + private [flink] def createBoundedProcessingOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType, + rowType: RowTypeInfo, + lowerBound: Int): ProcessFunction [Row, Row] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = true) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverRowProcessFunction( + aggregates, + aggFields, lowerBound, — End diff – `aggFields, lowerBound,` -> ` aggFields,` `lowerBound,`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108597606

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala —
          @@ -0,0 +1,157 @@
          +/*
          + * 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 org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.

          { Collector, Preconditions }

          +import org.apache.flink.api.common.state.ValueStateDescriptor
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.api.common.state.ValueState
          +import org.apache.flink.table.functions.

          { Accumulator, AggregateFunction }

          +import scala.collection.mutable.Queue
          +import org.apache.flink.api.common.state.ListStateDescriptor
          +import org.apache.flink.api.common.state.ListState
          +import org.apache.flink.api.common.typeinfo.TypeHint
          +import org.apache.flink.api.common.state.MapState
          +import org.apache.flink.api.common.state.MapStateDescriptor
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.api.java.typeutils.ListTypeInfo
          +import java.util.

          { ArrayList, LinkedList, List => JList }

          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
          +
          +class BoundedProcessingOverRowProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val bufferSize: Int,
          + private val forwardedFieldCount: Int,
          + private val aggregatesTypeInfo: RowTypeInfo,
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          + Preconditions.checkArgument(bufferSize > 0)
          +
          + private var accumulators: Row = _
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          +
          + override def open(config: Configuration)

          { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val currentTime = ctx.timerService().currentProcessingTime()
          + var i = 0
          +
          + var accumulators = accumulatorState.value()
          + // initialize state for the first processed element
          + if(accumulators == null){
          + accumulators = new Row(aggregates.length)
          + while (i < aggregates.length)

          { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + }
          +
          + val keyIter = rowMapState.keys.iterator
          + var oldestTimeStamp = currentTime
          + var toRetract: JList[Row] = null
          — End diff –

          Suggestion:
          `oldestTimeStamp -> retractTs`
          `toRetract-> retractList`
          What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108597606 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala — @@ -0,0 +1,157 @@ +/* + * 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 org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.types.Row +import org.apache.flink.util. { Collector, Preconditions } +import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.functions. { Accumulator, AggregateFunction } +import scala.collection.mutable.Queue +import org.apache.flink.api.common.state.ListStateDescriptor +import org.apache.flink.api.common.state.ListState +import org.apache.flink.api.common.typeinfo.TypeHint +import org.apache.flink.api.common.state.MapState +import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import java.util. { ArrayList, LinkedList, List => JList } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo + +class BoundedProcessingOverRowProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val bufferSize: Int, + private val forwardedFieldCount: Int, + private val aggregatesTypeInfo: RowTypeInfo, + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + Preconditions.checkArgument(bufferSize > 0) + + private var accumulators: Row = _ + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + + override def open(config: Configuration) { + + output = new Row(forwardedFieldCount + aggregates.length) + // We keep the elements received in a list state + // together with the ingestion time in the operator + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo) + + accumulatorState = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val currentTime = ctx.timerService().currentProcessingTime() + var i = 0 + + var accumulators = accumulatorState.value() + // initialize state for the first processed element + if(accumulators == null){ + accumulators = new Row(aggregates.length) + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val keyIter = rowMapState.keys.iterator + var oldestTimeStamp = currentTime + var toRetract: JList [Row] = null — End diff – Suggestion: `oldestTimeStamp -> retractTs` `toRetract-> retractList` What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user stefanobortoli commented on the issue:

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

          @fhueske @sunjincheng121 @rtudoran I have just completed the implementation with the MapState, please have a look.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske @sunjincheng121 @rtudoran I have just completed the implementation with the MapState, please have a look.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108398986

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("partitionBy", "c"),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          ok, now it works as you suggested

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108398986 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("partitionBy", "c"), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – ok, now it works as you suggested
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108378237

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("partitionBy", "c"),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          Have a look at this [test](https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala#L345).

          In fact, the required changes are already merged to master. So you would only need to rebase to get the value of the bound.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108378237 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("partitionBy", "c"), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – Have a look at this [test] ( https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala#L345 ). In fact, the required changes are already merged to master. So you would only need to rebase to get the value of the bound.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108372957

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("partitionBy", "c"),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          if I add the value the test wont pass. That is how the query is parsed in Calcite. Constant have to be resolved to get the lower boundary. Not sure I got your point.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108372957 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("partitionBy", "c"), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – if I add the value the test wont pass. That is how the query is parsed in Calcite. Constant have to be resolved to get the lower boundary. Not sure I got your point.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r108369391

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

          { + output = new Row(forwardedFieldCount + numAggregates) + }

          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          — End diff –

          extend and implement are different in java, really, just scala confusion to me.

          Show
          githubbot ASF GitHub Bot added a comment - Github user stefanobortoli commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r108369391 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class IncrementalAggregateOverWindowFunction [W <: Window] ( + private val numGroupingKey: Int, + private val numAggregates: Int, + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + + private var output: Row = _ + private var reuse: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + numAggregates) + } + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next — End diff – extend and implement are different in java, really, just scala confusion to me.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          I agree @huawei-flink, an efficient order-preserving state primitive (generic or specific for time) would be very helpful.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 I agree @huawei-flink, an efficient order-preserving state primitive (generic or specific for time) would be very helpful.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske Thanks a lot of the clarification. I understand the issue better now, and see your attempt to make an average case that would work for both in memory as well as on external persistence. Considering RocksDB as the state of art, your choice sounds much more reasonable. We are well aware of the costs of serialization, and the impact is definitely important. However, low latency systems with strict SLA will likely run just in memory.

          The O of the MapState is granted by the fact that time is monothonic and therefore the sequential reading is managed by the key timestamp. The cost of each O(1) in the hashmap increseas with the size of the window thou as you need to search through the map index. We definitely need better data access patterns for the state of "time series" types of data.

          I will try to internalize it and provide the MapState implementation

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske Thanks a lot of the clarification. I understand the issue better now, and see your attempt to make an average case that would work for both in memory as well as on external persistence. Considering RocksDB as the state of art, your choice sounds much more reasonable. We are well aware of the costs of serialization, and the impact is definitely important. However, low latency systems with strict SLA will likely run just in memory. The O of the MapState is granted by the fact that time is monothonic and therefore the sequential reading is managed by the key timestamp. The cost of each O(1) in the hashmap increseas with the size of the window thou as you need to search through the map index. We definitely need better data access patterns for the state of "time series" types of data. I will try to internalize it and provide the MapState implementation
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @huawei-flink, thanks for your detailed explanation.

          The benefits of the MapState are that we only need to deserialize all keys and not all rows as in the ValueState or ListState case. Identifying the smallest key (as needed for OVER ROWS) is basically for free. Once the smallest key has been found, we only need to deserialize the rows that need to be retracted. All other rows are not touched at all.

          The benchmarks that @rtudoran ran were done with an in-memory state backend, which does not de/serialize data but keeps the state as objects on the heap. I think the numbers would be different if you would switch to the RocksDB state backend which serializes all data (RocksDB is the only state backend recommended for production settings). In fact, I would read from the result of the benchmarks that sorting the keys does not have a major impact on the performance. Another important aspect of the design is that RocksDB iterates of the the map keys in order, so even sorting (or rather ensuring a sorted order) becomes O.

          I do see the benefits of keeping data in order, but de/serialization is one of the major costs when processing data on the JVM and it makes a lot of sense to optimize for reduced de/serialization overhead.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @huawei-flink, thanks for your detailed explanation. The benefits of the MapState are that we only need to deserialize all keys and not all rows as in the ValueState or ListState case. Identifying the smallest key (as needed for OVER ROWS) is basically for free. Once the smallest key has been found, we only need to deserialize the rows that need to be retracted. All other rows are not touched at all. The benchmarks that @rtudoran ran were done with an in-memory state backend, which does not de/serialize data but keeps the state as objects on the heap. I think the numbers would be different if you would switch to the RocksDB state backend which serializes all data (RocksDB is the only state backend recommended for production settings). In fact, I would read from the result of the benchmarks that sorting the keys does not have a major impact on the performance. Another important aspect of the design is that RocksDB iterates of the the map keys in order, so even sorting (or rather ensuring a sorted order) becomes O . I do see the benefits of keeping data in order, but de/serialization is one of the major costs when processing data on the JVM and it makes a lot of sense to optimize for reduced de/serialization overhead.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          Hi @fhueske, @sunjincheng121 ,

          let me try to explain my perspective on this specific case (row based, proc time). This is for the purpose of discussion, to show that we are spending thoughts on this topic for a while now.

          In case of the row range, the "serialization savings" coming from MapState exists up to the point in which the "buffer" is filled. After that that, we need to start retracting to keep the value correct and to do that, we need to deserialize all the objects. as @rtudoran mentioned, we implemented a version using a Queue object.

          This has many advantages:

          • removing the object from the buffer at the right moment freeing memory on the go (without any iteration over the key set)
          • has the data access pattern of O(1) without any "key resolution costs" and no list iteration
          • keeps the natural processing order by design, without the need of indexing objects with timestamps
          • the experiments we run show that there are no difference for windows up to 100k elements, and after that the queue seems to be more efficient (as the the key resolution does not come for free).

          The map state may have a slight advantage in the early stages, when the window is not filled, but after it just introduces useless operations. Furthermore, the need to index objects with a created timestamp (more memory wasted), dealing with a sequential access (List) to get the most recent object when you can actually just use the natural arrival order seems useless complication. Applying the Occam Razor there should be no doubt on which solution we should be selecting first. The serialization optimization while window gets filled sounds like a premature optimization not worth in the long run. The further implementation of SQL operators (e.g. LIMIT, OFFSET etc) can just benefit from the fact that the state is already sorted, whereas the map would need to be sorted all the time.

          Of course I am talking specifically of the procTime semantic operations. eventTime is another story anyway. The map state as minor advantages in the beginning (as anyway the serialization costs are small), the queue state as advantages in executions running steadily because of access pattern and natural buffer cleansing.

          These are my two cents on the discussion

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 Hi @fhueske, @sunjincheng121 , let me try to explain my perspective on this specific case (row based, proc time). This is for the purpose of discussion, to show that we are spending thoughts on this topic for a while now. In case of the row range, the "serialization savings" coming from MapState exists up to the point in which the "buffer" is filled. After that that, we need to start retracting to keep the value correct and to do that, we need to deserialize all the objects. as @rtudoran mentioned, we implemented a version using a Queue object. This has many advantages: removing the object from the buffer at the right moment freeing memory on the go (without any iteration over the key set) has the data access pattern of O(1) without any "key resolution costs" and no list iteration keeps the natural processing order by design, without the need of indexing objects with timestamps the experiments we run show that there are no difference for windows up to 100k elements, and after that the queue seems to be more efficient (as the the key resolution does not come for free). The map state may have a slight advantage in the early stages, when the window is not filled, but after it just introduces useless operations. Furthermore, the need to index objects with a created timestamp (more memory wasted), dealing with a sequential access (List) to get the most recent object when you can actually just use the natural arrival order seems useless complication. Applying the Occam Razor there should be no doubt on which solution we should be selecting first. The serialization optimization while window gets filled sounds like a premature optimization not worth in the long run. The further implementation of SQL operators (e.g. LIMIT, OFFSET etc) can just benefit from the fact that the state is already sorted, whereas the map would need to be sorted all the time. Of course I am talking specifically of the procTime semantic operations. eventTime is another story anyway. The map state as minor advantages in the beginning (as anyway the serialization costs are small), the queue state as advantages in executions running steadily because of access pattern and natural buffer cleansing. These are my two cents on the discussion
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @huawei-flink, let me explain the idea of using `MapState` and its benefits in more detail.

          I'll start with the way that a `ListState` works. With `ListState` we can get efficient access to the head element of the list. However, when updating the `ListState`, we cannot remove individual elements but have to clear the complete state and reinsert all elements that should remain. Hence we always need to deserialize and serialize all elements of a `ListState`.

          With the `MapState` approach, we would put the elements in a map which is keyed on their processing timestamp. Since multiple records can arrive within the same millisecond, we use a `List[Row]` as value type for the map. To process a new row, we have to find the "oldest" row (i.e., the one with the smallest timestamp) to retract it from the accumulator. With `ListState` this is trivial, it is the head element. With `MapState` we have to iterate over the keys and find the smallest one (smallest processing timestamp). This requires to deserialize all keys, but these are only `Long` values and not complete rows. With the smallest key, we can get the `List[Row]` value and take the first Row from the list and retract it from the accumulator. When updating the state, we only update the `List[Row]` value of the smallest key (or possible remove it if the `List[Row]` became empty).

          So the benefit of using `MapState` of `ListState` is that we only read `n` Long (+ read/write 1 `List[Row]`) instead of reading and writing `n` Row values.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @huawei-flink, let me explain the idea of using `MapState` and its benefits in more detail. I'll start with the way that a `ListState` works. With `ListState` we can get efficient access to the head element of the list. However, when updating the `ListState`, we cannot remove individual elements but have to clear the complete state and reinsert all elements that should remain. Hence we always need to deserialize and serialize all elements of a `ListState`. With the `MapState` approach, we would put the elements in a map which is keyed on their processing timestamp. Since multiple records can arrive within the same millisecond, we use a `List [Row] ` as value type for the map. To process a new row, we have to find the "oldest" row (i.e., the one with the smallest timestamp) to retract it from the accumulator. With `ListState` this is trivial, it is the head element. With `MapState` we have to iterate over the keys and find the smallest one (smallest processing timestamp). This requires to deserialize all keys, but these are only `Long` values and not complete rows. With the smallest key, we can get the `List [Row] ` value and take the first Row from the list and retract it from the accumulator. When updating the state, we only update the `List [Row] ` value of the smallest key (or possible remove it if the `List [Row] ` became empty). So the benefit of using `MapState` of `ListState` is that we only read `n` Long (+ read/write 1 `List [Row] `) instead of reading and writing `n` Row values.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi, @fhueske Sounds good.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3574 Hi, @fhueske Sounds good.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @sunjincheng121, I like the idea of using `MapState` because we do not need to deserialize and serialize the Rows when updating the state as in case of a `ValueState` or `ListState` but just the Long timestamps to identify the Row to retract which should be OK.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @sunjincheng121, I like the idea of using `MapState` because we do not need to deserialize and serialize the Rows when updating the state as in case of a `ValueState` or `ListState` but just the Long timestamps to identify the Row to retract which should be OK.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi, @fhueske @huawei-flink about processing OVER ROW PRECEDING windows I implemented a version in #3585 . according to my version of the realization, with 90% similarity. Can you take a look and then discuss together.

          Best,
          SunJincheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3574 Hi, @fhueske @huawei-flink about processing OVER ROW PRECEDING windows I implemented a version in #3585 . according to my version of the realization, with 90% similarity. Can you take a look and then discuss together. Best, SunJincheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @huawei-flink, all of the current aggregation functions (including avg) should support retraction.
          Have a look at https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @huawei-flink, all of the current aggregation functions (including avg) should support retraction. Have a look at https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          I have a first implementation of the processFunction, using a Queue as a state. However, I need to implement the retractableAggregation, as AVG for example is not supported.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 I have a first implementation of the processFunction, using a Queue as a state. However, I need to implement the retractableAggregation, as AVG for example is not supported.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @sunjincheng121, I think processing `OVER ROW PRECEDING` windows are a much simpler case than the event-time variant. Not sure how much code could be shared.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @sunjincheng121, I think processing `OVER ROW PRECEDING` windows are a much simpler case than the event-time variant. Not sure how much code could be shared.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi, @huawei-flink @fhueske I have used `ProcessFunction` and implemented bounded event-time OVER window. Since I think both bounded proc-time can share the same infra. So I have completed the bounded proc-time OVER window design also in the PR #3585, Please take a look.

          Best,
          SunJIncheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3574 Hi, @huawei-flink @fhueske I have used `ProcessFunction` and implemented bounded event-time OVER window. Since I think both bounded proc-time can share the same infra. So I have completed the bounded proc-time OVER window design also in the PR #3585, Please take a look. Best, SunJIncheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi, @huawei-flink @fhueske the PR #3585 had include the logic , we can discuss together.
          What do you think?
          Thanks,
          SunJincheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3574 Hi, @huawei-flink @fhueske the PR #3585 had include the logic , we can discuss together. What do you think? Thanks, SunJincheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @huawei-flink,

          We would go for a `ProcessFunction` and implement it as follows:

          • `open()` In open we create two types of state: 1) the accumulators, which is a `ValueState[Row]`, 2) the rows we need to retract. This can be a `ListState[Row]`.
          • `processElement()` is called for each arriving row. We accumulate the row to the accumulator (fetched from the state) and retract the oldest value in the list of values to retract. Since the list is sorted, we take the head element. We emit the row with the updated accumulator and append it to the list of rows to retract.
          • `onTimer()` is not required for processing time OVER window.

          If we want to make it more efficient, we could just put those fields into the retraction rows which are actually used by the aggregation function and not the complete row.

          What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3574 Hi @huawei-flink, We would go for a `ProcessFunction` and implement it as follows: `open()` In open we create two types of state: 1) the accumulators, which is a `ValueState [Row] `, 2) the rows we need to retract. This can be a `ListState [Row] `. `processElement()` is called for each arriving row. We accumulate the row to the accumulator (fetched from the state) and retract the oldest value in the list of values to retract. Since the list is sorted, we take the head element. We emit the row with the updated accumulator and append it to the list of rows to retract. `onTimer()` is not required for processing time OVER window. If we want to make it more efficient, we could just put those fields into the retraction rows which are actually used by the aggregation function and not the complete row. What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107151292

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          If you change the query to `10 PRECEDING` the test will still pass. The `$3` is the index not the value of the constant.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107151292 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – If you change the query to `10 PRECEDING` the test will still pass. The `$3` is the index not the value of the constant.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107150862

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

          { + output = new Row(forwardedFieldCount + numAggregates) + }

          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          — End diff –

          actually it is implementing two interfaces which is also supported by Java

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107150862 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class IncrementalAggregateOverWindowFunction [W <: Window] ( + private val numGroupingKey: Int, + private val numAggregates: Int, + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + + private var output: Row = _ + private var reuse: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + numAggregates) + } + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next — End diff – actually it is implementing two interfaces which is also supported by Java
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107150669

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -106,6 +112,10 @@ class DataStreamOverAggregate(
          if (overWindow.lowerBound.isUnbounded &&
          overWindow.upperBound.isCurrentRow)

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a RowType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.isRows &&
          + (overWindow.upperBound.isPreceding || overWindow.upperBound.isCurrentRow)) {
          — End diff –

          If the upper bound is not specified the default is `CURRENT ROW`. So this check can be safely added, IMO. Just debug the case of `OVER (ORDER BY procTime() ROWS 2 PRECEDING)`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107150669 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,6 +112,10 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a RowType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.isRows && + (overWindow.upperBound.isPreceding || overWindow.upperBound.isCurrentRow)) { — End diff – If the upper bound is not specified the default is `CURRENT ROW`. So this check can be safely added, IMO. Just debug the case of `OVER (ORDER BY procTime() ROWS 2 PRECEDING)`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @sunjincheng121 @fhueske as I understand, the open function is called for every element. So the accumulator is retrieved based on the state. So, the supportQueue is created in the constructor, and the open function just retrieves the latest state. Then the process function retracts the value evicted and accumulate the new one. Then the state is updated. And so on. I will give it a try.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 @sunjincheng121 @fhueske as I understand, the open function is called for every element. So the accumulator is retrieved based on the state. So, the supportQueue is created in the constructor, and the open function just retrieves the latest state. Then the process function retracts the value evicted and accumulate the new one. Then the state is updated. And so on. I will give it a try.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107146512

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
              • End diff –

          Yes, your right.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107146512 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( End diff – Yes, your right.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske @sunjincheng121 would a ProcessWindowFunction work?

          or, using just the ProcessFunction, how would I keep track of the lowerbound? I was thinking of using a counter to be checked against the lowerbound. How does the scan work in the stream? from oldest to earliest? or the other way around? How do I get the relative "stream position" to consider the lower bounds when processing a single element? when is the ProcessFunction "open"?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske @sunjincheng121 would a ProcessWindowFunction work? or, using just the ProcessFunction, how would I keep track of the lowerbound? I was thinking of using a counter to be checked against the lowerbound. How does the scan work in the stream? from oldest to earliest? or the other way around? How do I get the relative "stream position" to consider the lower bounds when processing a single element? when is the ProcessFunction "open"?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske @sunjincheng121 I see your points related to the window checkpointing. I start also to see a problem with the overall design based on ProcessFunction which make window object obsolete. FLINK-5572 proposes a solution to avoid memory duplication in Window which in my opinion are better tools. Perhaps we could look into that direction to improve memory efficiency.

          To be honest, I don't even see the need for eager aggregation, as the "between X and current row" will not include millions of element, but it is likely to be used with relatively small ranges (10^2?).

          Nevertheless, I will look into the ProcessFunction although I have feeling we are making our way through the overall SQL implementation more complex as we move complexity in the aggregation rather than window semantics. Of course these are my2c, and if ProcessFunction works better I will implement it. However I have the feeling that we are just transforming the aggregation into windows, nesting granular state management when windows check-pointing can manage everything smoothly and in a very elegant way.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske @sunjincheng121 I see your points related to the window checkpointing. I start also to see a problem with the overall design based on ProcessFunction which make window object obsolete. FLINK-5572 proposes a solution to avoid memory duplication in Window which in my opinion are better tools. Perhaps we could look into that direction to improve memory efficiency. To be honest, I don't even see the need for eager aggregation, as the "between X and current row" will not include millions of element, but it is likely to be used with relatively small ranges (10^2?). Nevertheless, I will look into the ProcessFunction although I have feeling we are making our way through the overall SQL implementation more complex as we move complexity in the aggregation rather than window semantics. Of course these are my2c, and if ProcessFunction works better I will implement it. However I have the feeling that we are just transforming the aggregation into windows, nesting granular state management when windows check-pointing can manage everything smoothly and in a very elegant way.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107107833

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +95,18 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + def getLowerBoundary(
          + constants: ImmutableList[RexLiteral],
          + lowerBound: RexWindowBound,
          + input: RelNode): Int = {
          + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf[RexInputRef]
          + val index:Int = ref.getIndex
          + val count: Int = input.getRowType.getFieldCount
          + val lowerBoundIndex = count - index;
          — End diff –

          rm `;`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107107833 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +95,18 @@ object AggregateUtil { aggregationStateType) } } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode): Int = { + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf [RexInputRef] + val index:Int = ref.getIndex + val count: Int = input.getRowType.getFieldCount + val lowerBoundIndex = count - index; — End diff – rm `;`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107107801

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +95,18 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + def getLowerBoundary(
          + constants: ImmutableList[RexLiteral],
          + lowerBound: RexWindowBound,
          + input: RelNode): Int = {
          + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf[RexInputRef]
          + val index:Int = ref.getIndex
          — End diff –

          +space

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107107801 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +95,18 @@ object AggregateUtil { aggregationStateType) } } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode): Int = { + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf [RexInputRef] + val index:Int = ref.getIndex — End diff – +space
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107107750

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
              • End diff –

          This just gives the index that needs to be looked up but not the value of the range.
          If I get the logic right, the refers to the input fields if it is `< getFieldCount` and to constants if it is `>= getFieldCount`.
          I looked into this computation and I think it is correct.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107107750 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( End diff – This just gives the index that needs to be looked up but not the value of the range. If I get the logic right, the refers to the input fields if it is `< getFieldCount` and to constants if it is `>= getFieldCount`. I looked into this computation and I think it is correct.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3574#discussion_r107107006

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +95,18 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + def getLowerBoundary(
          + constants: ImmutableList[RexLiteral],
          + lowerBound: RexWindowBound,
          + input: RelNode): Int = {
          + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf[RexInputRef]
          + val index:Int = ref.getIndex
          + val count: Int = input.getRowType.getFieldCount
          — End diff –

          The offset must be used to get the right position constants array. If there are more constants in the query, I need to consider the overall field count and subtract the offset to get the right position in the constants array. To be honest, this part is quite obscure, and also on Calcite mailing list the reply I got was quite confusing and wrong (in practice).

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107107006 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +95,18 @@ object AggregateUtil { aggregationStateType) } } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode): Int = { + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf [RexInputRef] + val index:Int = ref.getIndex + val count: Int = input.getRowType.getFieldCount — End diff – The offset must be used to get the right position constants array. If there are more constants in the query, I need to consider the overall field count and subtract the offset to get the right position in the constants array. To be honest, this part is quite obscure, and also on Calcite mailing list the reply I got was quite confusing and wrong (in practice).
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3574#discussion_r107105500

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
              • End diff –

          I will check, but your suggestion seems wrong. The index needs to be solved in the constants array.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107105500 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( End diff – I will check, but your suggestion seems wrong. The index needs to be solved in the constants array.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3574#discussion_r107104785

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          what do you mean? The range value arrives like that, with the value presented in the constants array.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107104785 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – what do you mean? The range value arrives like that, with the value presented in the constants array.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3574#discussion_r107103813

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

          { + output = new Row(forwardedFieldCount + numAggregates) + }

          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          — End diff –

          I agree with you, the logic is exactly the same. Wanted to be sure the PR was "correct" first. I am not used to the fact that SCALA allows for multiple class extension. :-P

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107103813 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class IncrementalAggregateOverWindowFunction [W <: Window] ( + private val numGroupingKey: Int, + private val numAggregates: Int, + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + + private var output: Row = _ + private var reuse: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + numAggregates) + } + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next — End diff – I agree with you, the logic is exactly the same. Wanted to be sure the PR was "correct" first. I am not used to the fact that SCALA allows for multiple class extension. :-P
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3574#discussion_r107103378

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput()) + 1
            +
            + val (aggFunction, accumulatorRowType, aggResultRowType) =
              • End diff –

          you are right, they become the same in this case.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107103378 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + 1 + + val (aggFunction, accumulatorRowType, aggResultRowType) = End diff – you are right, they become the same in this case.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3574#discussion_r107103196

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -106,6 +112,10 @@ class DataStreamOverAggregate(
          if (overWindow.lowerBound.isUnbounded &&
          overWindow.upperBound.isCurrentRow)

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a RowType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.isRows &&
          + (overWindow.upperBound.isPreceding || overWindow.upperBound.isCurrentRow)) {
          — End diff –

          I missed the "not unbounded" clause. However, this check works also for the queries: " BETWEEN 2 ROWS PRECEDING" - letting implicit the current row. Don't you think that being a little flexible on syntax could be useful? If you want to be strict on this, I will apply the check logic as you suggested.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107103196 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,6 +112,10 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a RowType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.isRows && + (overWindow.upperBound.isPreceding || overWindow.upperBound.isCurrentRow)) { — End diff – I missed the "not unbounded" clause. However, this check works also for the queries: " BETWEEN 2 ROWS PRECEDING" - letting implicit the current row. Don't you think that being a little flexible on syntax could be useful? If you want to be strict on this, I will apply the check logic as you suggested.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107066582

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
              • End diff –

          How about using `lowerbound = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex` ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107066582 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( End diff – How about using `lowerbound = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex` ?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r107066745

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +95,18 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + def getLowerBoundary(
          + constants: ImmutableList[RexLiteral],
          + lowerBound: RexWindowBound,
          + input: RelNode): Int = {
          + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf[RexInputRef]
          + val index:Int = ref.getIndex
          + val count: Int = input.getRowType.getFieldCount
          — End diff –

          Can you explain more why we using `input.getRowType.getFieldCount` to calculation the LowerBoundary?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107066745 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +95,18 @@ object AggregateUtil { aggregationStateType) } } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode): Int = { + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf [RexInputRef] + val index:Int = ref.getIndex + val count: Int = input.getRowType.getFieldCount — End diff – Can you explain more why we using `input.getRowType.getFieldCount` to calculation the LowerBoundary?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106997276

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          Can we add the actual range value here instead of `$3`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106997276 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – Can we add the actual range value here instead of `$3`?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106958710

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput()) + 1
            +
            + val (aggFunction, accumulatorRowType, aggResultRowType) =
            + AggregateUtil.createDataStreamOverAggregateFunction(
            + namedAggregates,
            + inputType,
            + rowRelDataType,
            + partitionKeys)
            + val aggString = aggregationToString(
              • End diff –

          `aggString` is not used.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106958710 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + 1 + + val (aggFunction, accumulatorRowType, aggResultRowType) = + AggregateUtil.createDataStreamOverAggregateFunction( + namedAggregates, + inputType, + rowRelDataType, + partitionKeys) + val aggString = aggregationToString( End diff – `aggString` is not used.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106964910

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala —
          @@ -0,0 +1,102 @@
          +/*
          + * 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.util.

          {ArrayList => JArrayList, List => JList}

          +import org.apache.flink.api.common.functions.

          {AggregateFunction => DataStreamAggOverFunc}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +/**
          + * Aggregate Function used for the aggregate operator in
          + * [[org.apache.flink.streaming.api.datastream.WindowedStream]]
          + *
          + * @param aggregates the list of all [[org.apache.flink.table.functions.AggregateFunction]]
          + * used for this aggregation
          + * @param aggFields the position (in the input Row) of the input value for each aggregate
          + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation
          + */
          +class AggregateAggOverFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends DataStreamAggOverFunc[Row, Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var lastVal:Row = _
          — End diff –

          +space `lastVal: Row`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106964910 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * 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.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ — End diff – +space `lastVal: Row`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106963715

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput()) + 1
            +
            + val (aggFunction, accumulatorRowType, aggResultRowType) =
              • End diff –

          `aggResultRowType` and `rowTypeInfo` should be the same. Please add a safety check for this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106963715 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + 1 + + val (aggFunction, accumulatorRowType, aggResultRowType) = End diff – `aggResultRowType` and `rowTypeInfo` should be the same. Please add a safety check for this.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106996212

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -317,4 +317,465 @@ class SqlITCase extends StreamingWithStateTestBase

          { result.addSink(new StreamITCase.StringSink) env.execute() }

          +
          + /**
          + *
          + * //////////////////////////////////////////////////////
          + * START TESTING BOUNDED PROC TIME ROW AGGREGATIO
          + * //////////////////////////////////////////////////////
          + *
          + */
          +
          + @Test
          + def testUnpartitionedMaxAggregatation(): Unit = {
          — End diff –

          Many of these tests test the same feature with different aggregation functions.
          Since the aggregation functions are separately tested, I'd suggest to restrict the tests to four different configurations of OVER windows:

          1. multiple aggregations, partitioned, 2 rows preceding
          2. multiple aggregations, partitioned, 4 rows preceding
          3. multiple aggregations, non-partitioned, 2 rows preceding
          4. multiple aggregations, non-partitioned, 10 rows preceding

          This should give us good coverage of the feature.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106996212 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +317,465 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** + * + * ////////////////////////////////////////////////////// + * START TESTING BOUNDED PROC TIME ROW AGGREGATIO + * ////////////////////////////////////////////////////// + * + */ + + @Test + def testUnpartitionedMaxAggregatation(): Unit = { — End diff – Many of these tests test the same feature with different aggregation functions. Since the aggregation functions are separately tested, I'd suggest to restrict the tests to four different configurations of OVER windows: 1. multiple aggregations, partitioned, 2 rows preceding 2. multiple aggregations, partitioned, 4 rows preceding 3. multiple aggregations, non-partitioned, 2 rows preceding 4. multiple aggregations, non-partitioned, 10 rows preceding This should give us good coverage of the feature.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106962344

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

          { + output = new Row(forwardedFieldCount + numAggregates) + }

          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          — End diff –

          This function copies each field of the intermediate row into the output row.

          I think we can merge `IncrementalAggregateOverWindowFunction` and `IncrementalAggregateOverAllWindowFunction` into a class called `ForwardWindowFunction` which is defined as below:

          ```
          class ForwardWindowFunction[W <: Window]()
          extends WindowFunction[Row, Row, Tuple, W] with AllWindowFunction[Row, Row, W] {

          override def apply(
          key: Tuple,
          window: W,
          records: Iterable[Row],
          out: Collector[Row]): Unit = {

          val iter = records.iterator
          while (iter.hasNext)

          { out.collect(iter.next) }
          }

          override def apply(window: W, records: Iterable[Row], out: Collector[Row]): Unit = {

          val iter = records.iterator
          while (iter.hasNext) { out.collect(iter.next) }

          }
          }
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106962344 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala — @@ -0,0 +1,73 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class IncrementalAggregateOverWindowFunction [W <: Window] ( + private val numGroupingKey: Int, + private val numAggregates: Int, + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + + private var output: Row = _ + private var reuse: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + numAggregates) + } + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next — End diff – This function copies each field of the intermediate row into the output row. I think we can merge `IncrementalAggregateOverWindowFunction` and `IncrementalAggregateOverAllWindowFunction` into a class called `ForwardWindowFunction` which is defined as below: ``` class ForwardWindowFunction [W <: Window] () extends WindowFunction [Row, Row, Tuple, W] with AllWindowFunction [Row, Row, W] { override def apply( key: Tuple, window: W, records: Iterable [Row] , out: Collector [Row] ): Unit = { val iter = records.iterator while (iter.hasNext) { out.collect(iter.next) } } override def apply(window: W, records: Iterable [Row] , out: Collector [Row] ): Unit = { val iter = records.iterator while (iter.hasNext) { out.collect(iter.next) } } } ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106957084

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -150,12 +160,71 @@ class DataStreamOverAggregate(
          inputType,
          false)

          • inputDS
          • .process(processFunction).setParallelism(1).setMaxParallelism(1)
          • .returns(rowTypeInfo)
          • .name(aggOpName)
          • .asInstanceOf[DataStream[Row]]
          • }
            + inputDS
            + .process(processFunction).setParallelism(1).setMaxParallelism(1)
            + .returns(rowTypeInfo)
            + .name(aggOpName)
            + .asInstanceOf[DataStream[Row]]
            + }
              • End diff –

          This indention change is good, IMO.
          Can you also fix the indention of the whole `if else` block? Thanks

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106957084 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -150,12 +160,71 @@ class DataStreamOverAggregate( inputType, false) inputDS .process(processFunction).setParallelism(1).setMaxParallelism(1) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream [Row] ] + } End diff – This indention change is good, IMO. Can you also fix the indention of the whole `if else` block? Thanks
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106966705

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala —
          @@ -0,0 +1,102 @@
          +/*
          + * 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.util.

          {ArrayList => JArrayList, List => JList}

          +import org.apache.flink.api.common.functions.

          {AggregateFunction => DataStreamAggOverFunc}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +/**
          + * Aggregate Function used for the aggregate operator in
          + * [[org.apache.flink.streaming.api.datastream.WindowedStream]]
          + *
          + * @param aggregates the list of all [[org.apache.flink.table.functions.AggregateFunction]]
          + * used for this aggregation
          + * @param aggFields the position (in the input Row) of the input value for each aggregate
          + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation
          + */
          +class AggregateAggOverFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends DataStreamAggOverFunc[Row, Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var lastVal:Row = _
          +
          + override def createAccumulator(): Row = {
          + val accumulatorRow: Row = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

          { + accumulatorRow.setField(i, aggregates(i).createAccumulator()) + i += 1 + }

          + accumulatorRow
          + }
          +
          + override def add(value: Row, accumulatorRow: Row): Unit = {
          + var i = 0
          + while (i < aggregates.length)

          { + val acc = accumulatorRow.getField(i).asInstanceOf[Accumulator] + val v = value.getField(aggFields(i)) + aggregates(i).accumulate(acc, v) + i += 1 + }

          + lastVal = value;
          + }
          +
          + override def getResult(accumulatorRow: Row): Row = {
          + val output = new Row(forwardedFieldCount + aggFields.length)
          +
          + var i = 0
          + // set the output value of forward fields
          + while (i < forwardedFieldCount)

          { + output.setField(i, lastVal.getField(i)) + i += 1 + }

          +
          + i = 0
          + while (i < aggregates.length)

          { + val acc = accumulatorRow.getField(i).asInstanceOf[Accumulator] + output.setField(forwardedFieldCount + i, aggregates(i).getValue(acc)) + i += 1 + }

          + output
          + }
          +
          + override def merge(aAccumulatorRow: Row, bAccumulatorRow: Row): Row = {
          +
          + var i = 0
          + while (i < aggregates.length) {
          — End diff –

          I'm sorry, I just realized that we cannot use the `AggregateFunction` interface.
          We would need to merge the forwarded fields as well. Since we do not know which accumulator holds the forwarded fields of the last row, it is not possible to use this interface. I'm sorry that I led you into this direction.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106966705 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * 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.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ + + override def createAccumulator(): Row = { + val accumulatorRow: Row = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulatorRow.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + accumulatorRow + } + + override def add(value: Row, accumulatorRow: Row): Unit = { + var i = 0 + while (i < aggregates.length) { + val acc = accumulatorRow.getField(i).asInstanceOf[Accumulator] + val v = value.getField(aggFields(i)) + aggregates(i).accumulate(acc, v) + i += 1 + } + lastVal = value; + } + + override def getResult(accumulatorRow: Row): Row = { + val output = new Row(forwardedFieldCount + aggFields.length) + + var i = 0 + // set the output value of forward fields + while (i < forwardedFieldCount) { + output.setField(i, lastVal.getField(i)) + i += 1 + } + + i = 0 + while (i < aggregates.length) { + val acc = accumulatorRow.getField(i).asInstanceOf[Accumulator] + output.setField(forwardedFieldCount + i, aggregates(i).getValue(acc)) + i += 1 + } + output + } + + override def merge(aAccumulatorRow: Row, bAccumulatorRow: Row): Row = { + + var i = 0 + while (i < aggregates.length) { — End diff – I'm sorry, I just realized that we cannot use the `AggregateFunction` interface. We would need to merge the forwarded fields as well. Since we do not know which accumulator holds the forwarded fields of the last row, it is not possible to use this interface. I'm sorry that I led you into this direction.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106965219

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala —
          @@ -0,0 +1,102 @@
          +/*
          + * 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.util.

          {ArrayList => JArrayList, List => JList}

          +import org.apache.flink.api.common.functions.

          {AggregateFunction => DataStreamAggOverFunc}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +/**
          + * Aggregate Function used for the aggregate operator in
          + * [[org.apache.flink.streaming.api.datastream.WindowedStream]]
          + *
          + * @param aggregates the list of all [[org.apache.flink.table.functions.AggregateFunction]]
          + * used for this aggregation
          + * @param aggFields the position (in the input Row) of the input value for each aggregate
          + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation
          + */
          +class AggregateAggOverFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends DataStreamAggOverFunc[Row, Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var lastVal:Row = _
          — End diff –

          We cannot save the last record in a member variable. This will be lost in case of a failure. We have to put it in the accumulator such that it is checkpointed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106965219 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * 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.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ — End diff – We cannot save the last record in a member variable. This will be lost in case of a failure. We have to put it in the accumulator such that it is checkpointed.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106965395

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala —
          @@ -0,0 +1,102 @@
          +/*
          + * 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.util.

          {ArrayList => JArrayList, List => JList}

          +import org.apache.flink.api.common.functions.

          {AggregateFunction => DataStreamAggOverFunc}

          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Preconditions
          +
          +/**
          + * Aggregate Function used for the aggregate operator in
          + * [[org.apache.flink.streaming.api.datastream.WindowedStream]]
          + *
          + * @param aggregates the list of all [[org.apache.flink.table.functions.AggregateFunction]]
          + * used for this aggregation
          + * @param aggFields the position (in the input Row) of the input value for each aggregate
          + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation
          + */
          +class AggregateAggOverFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends DataStreamAggOverFunc[Row, Row, Row] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var lastVal:Row = _
          +
          + override def createAccumulator(): Row = {
          + val accumulatorRow: Row = new Row(aggregates.length)
          — End diff –

          The accumulator row must include the forwarded fields to checkpoint the last received row.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106965395 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * 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.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ + + override def createAccumulator(): Row = { + val accumulatorRow: Row = new Row(aggregates.length) — End diff – The accumulator row must include the forwarded fields to checkpoint the last received row.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106993805

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -661,6 +679,36 @@ object AggregateUtil

          { (aggFunction, accumulatorRowType, aggResultRowType) }

          +
          +
          + private[flink] def createDataStreamOverAggregateFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType,
          + outputType: RelDataType,
          + groupKeysIndex: Array[Int])
          + : (DataStreamAggFunction[Row, Row, Row], RowTypeInfo, RowTypeInfo) = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregateMapping = getAggregateMapping(namedAggregates, outputType)
          — End diff –

          This is not used except for the following check. Can the check and `aggreagteMapping` be removed?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106993805 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -661,6 +679,36 @@ object AggregateUtil { (aggFunction, accumulatorRowType, aggResultRowType) } + + + private [flink] def createDataStreamOverAggregateFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType, + outputType: RelDataType, + groupKeysIndex: Array [Int] ) + : (DataStreamAggFunction [Row, Row, Row] , RowTypeInfo, RowTypeInfo) = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregateMapping = getAggregateMapping(namedAggregates, outputType) — End diff – This is not used except for the following check. Can the check and `aggreagteMapping` be removed?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106955390

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -106,6 +112,10 @@ class DataStreamOverAggregate(
          if (overWindow.lowerBound.isUnbounded &&
          overWindow.upperBound.isCurrentRow)

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a RowType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.isRows &&
          + (overWindow.upperBound.isPreceding || overWindow.upperBound.isCurrentRow)) {
          — End diff –

          I think the condition should be `overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow`.
          This will check that the lower bound in bounded and preceding and that the upper bound is the current row. The window implementation addresses exactly this case.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106955390 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,6 +112,10 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a RowType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.isRows && + (overWindow.upperBound.isPreceding || overWindow.upperBound.isCurrentRow)) { — End diff – I think the condition should be `overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow`. This will check that the lower bound in bounded and preceding and that the upper bound is the current row. The window implementation addresses exactly this case.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3574#discussion_r106997447

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala —
          @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

          { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + }

          +
          + @Test
          + def testBoundedPartitionedProcessingWindowWithRow() = {
          + val sql = "SELECT " +
          + "c, " +
          + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
          + "CURRENT ROW) as cnt1 " +
          + "from MyTable"
          +
          + val expected =
          + unaryNode(
          + "DataStreamCalc",
          + unaryNode(
          + "DataStreamOverAggregate",
          + unaryNode(
          + "DataStreamCalc",
          + streamTableNode(0),
          + term("select", "a", "c", "PROCTIME() AS $2")
          + ),
          + term("partitionBy", "c"),
          + term("orderBy", "PROCTIME"),
          + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"),
          — End diff –

          Can we add the actual range value here instead of `$3`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106997447 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +239,61 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testBoundedNonPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), + term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0") + ), + term("select", "c", "w0$o0 AS $1") + ) + streamUtil.verifySql(sql, expected) + } + + @Test + def testBoundedPartitionedProcessingWindowWithRow() = { + val sql = "SELECT " + + "c, " + + "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " + + "CURRENT ROW) as cnt1 " + + "from MyTable" + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamOverAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "PROCTIME() AS $2") + ), + term("partitionBy", "c"), + term("orderBy", "PROCTIME"), + term("rows", "BETWEEN $3 PRECEDING AND CURRENT ROW"), — End diff – Can we add the actual range value here instead of `$3`?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske let's see if the third attempt works. I have included all the comments of @sunjincheng121 (apart from the window function) and having done a merge on a fresh checkout also the imports should be fine.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske let's see if the third attempt works. I have included all the comments of @sunjincheng121 (apart from the window function) and having done a merge on a fresh checkout also the imports should be fine.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink closed the pull request at: https://github.com/apache/flink/pull/3547
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          sorry about the mess. I don't understand the mess the rebase does with eclipse... I will close this PR and open another one including all the changes and comments

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3547 sorry about the mess. I don't understand the mess the rebase does with eclipse... I will close this PR and open another one including all the changes and comments
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3547#discussion_r106917566

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -17,35 +17,62 @@
          */
          package org.apache.flink.table.runtime.aggregate

          -import java.util
          +import scala.collection.JavaConversions.asScalaBuffer
          — End diff –

          I need that import, without it does not build.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106917566 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -17,35 +17,62 @@ */ package org.apache.flink.table.runtime.aggregate -import java.util +import scala.collection.JavaConversions.asScalaBuffer — End diff – I need that import, without it does not build.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3547#discussion_r106868683

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala —
          @@ -0,0 +1,311 @@
          +/*
          + * 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.api.scala.stream.sql
          +
          +import org.apache.flink.api.scala._
          +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
          +import org.apache.flink.table.api.

          { TableEnvironment, TableException }

          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.api.scala.stream.utils.

          { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +}

          +import org.junit.Assert._
          +import org.junit._
          +import scala.collection.mutable
          +import org.apache.flink.types.Row
          +
          +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase {
          — End diff –

          @fhueske Should I add all the test unit under the SqlITCase class?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106868683 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala — @@ -0,0 +1,311 @@ +/* + * 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.api.scala.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api. { TableEnvironment, TableException } +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils. { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +} +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable +import org.apache.flink.types.Row + +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase { — End diff – @fhueske Should I add all the test unit under the SqlITCase class?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106627390

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,41 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.plan.

          { RelOptCluster, RelTraitSet }

          — End diff –

          No worries. I know that IDEs tend to reformat code but it really makes reviews harder.
          Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106627390 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,41 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.plan. { RelOptCluster, RelTraitSet } — End diff – No worries. I know that IDEs tend to reformat code but it really makes reviews harder. Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106627282

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -106,9 +113,14 @@ class DataStreamOverAggregate(
          if (overWindow.lowerBound.isUnbounded &&
          overWindow.upperBound.isCurrentRow)

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]
          — End diff –

          Yes, I realized that when looking at PR #3550 that `isInstanceOf[BasicSqlType]` and `.isInstanceOf[IntervalSqlType]` distinguishes ROW from RANGE windows. I think using `.isRows()` is more clear and might also be safer because it appears to be a more public API than the type of the offset.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106627282 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,9 +113,14 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] — End diff – Yes, I realized that when looking at PR #3550 that `isInstanceOf [BasicSqlType] ` and `.isInstanceOf [IntervalSqlType] ` distinguishes ROW from RANGE windows. I think using `.isRows()` is more clear and might also be safer because it appears to be a more public API than the type of the offset.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3547#discussion_r106616178

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -106,9 +113,14 @@ class DataStreamOverAggregate(
          if (overWindow.lowerBound.isUnbounded &&
          overWindow.upperBound.isCurrentRow)

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]
          — End diff –

          I guess that is also a way to do it. The check allows to distinguish between time bounded and row bounded. I have no particular affection for my solution, it just worked. I will apply and test yours as well.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106616178 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,9 +113,14 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] — End diff – I guess that is also a way to do it. The check allows to distinguish between time bounded and row bounded. I have no particular affection for my solution, it just worked. I will apply and test yours as well.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3547#discussion_r106615708

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,41 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.plan.

          { RelOptCluster, RelTraitSet }

          — End diff –

          @fhueske sorry about that. Will be more careful in the next one.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106615708 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,41 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.plan. { RelOptCluster, RelTraitSet } — End diff – @fhueske sorry about that. Will be more careful in the next one.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106615587

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
          +
          +class BoundedProcessingOverAllWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          + private var reuse: Row = _
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + if (null == accumulators)

          { + accumulators = new Row(aggregates.length) + }

          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + // setting the accumulators for each aggregation
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          What is the meaning of the acc?
          What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method
          should it be aggregates.resetAccumulator(aggregates)?

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106615587 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction + +class BoundedProcessingOverAllWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulators: Row = _ + private var reuse: Row = _ + + output = new Row(forwardedFieldCount + aggregates.length) + if (null == accumulators) { + accumulators = new Row(aggregates.length) + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + // setting the accumulators for each aggregation + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – What is the meaning of the acc? What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method should it be aggregates .resetAccumulator(aggregates )?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106615434

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
          +
          +class BoundedProcessingOverAllWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          + private var reuse: Row = _
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + if (null == accumulators)

          { + accumulators = new Row(aggregates.length) + }

          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + // setting the accumulators for each aggregation
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          What is the meaning of the acc?
          What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106615434 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction + +class BoundedProcessingOverAllWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulators: Row = _ + private var reuse: Row = _ + + output = new Row(forwardedFieldCount + aggregates.length) + if (null == accumulators) { + accumulators = new Row(aggregates.length) + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + // setting the accumulators for each aggregation + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – What is the meaning of the acc? What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106566112

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          Hi @huawei-flink @fhueske in bounded OVER ROW case, I still recommend using processFunction. But we can just agree to disagree at current time. Because in the continuous progress of flip11, we may change our view.
          Best,
          SunJIncheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106566112 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – Hi @huawei-flink @fhueske in bounded OVER ROW case, I still recommend using processFunction. But we can just agree to disagree at current time. Because in the continuous progress of flip11, we may change our view. Best, SunJIncheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          One more thing. The PR includes a merge commit which makes squashing the commits and merging the PR a lot more difficult (see also the [contribution guidelines](http://flink.apache.org/contribute-code.html#coding-guidelines)).

          You can avoid merge commits if you branch from the master and just put commits on top. If you want to include somebody else's work, you can use `git cherry-pick` to pull over a commit. Use `git rebase` to rebase your commits on the latest master.

          Best, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3547 One more thing. The PR includes a merge commit which makes squashing the commits and merging the PR a lot more difficult (see also the [contribution guidelines] ( http://flink.apache.org/contribute-code.html#coding-guidelines )). You can avoid merge commits if you branch from the master and just put commits on top. If you want to include somebody else's work, you can use `git cherry-pick` to pull over a commit. Use `git rebase` to rebase your commits on the latest master. Best, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106513895

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,41 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.plan.

          { RelOptCluster, RelTraitSet }

          — End diff –

          This PR has several changes that reformat the code but do not change the logic.
          In general, we try to limit PRs to the scope of the issue and avoid reformatting.

          1. It makes the PR harder to review because every reformatting change needs to be checked (was something changed or not). This is especially tiresome when indention is changed.
          2. It makes it more difficult to track changes in the history.
          3. Other users do not necessarily agree with your (or your IDE's) formatting and will format it back. So it might get back and forth.

          There is nothing wrong with adding a space between arguments here and there, but please keep reformatting changes to a minimum.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106513895 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,41 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.plan. { RelOptCluster, RelTraitSet } — End diff – This PR has several changes that reformat the code but do not change the logic. In general, we try to limit PRs to the scope of the issue and avoid reformatting. 1. It makes the PR harder to review because every reformatting change needs to be checked (was something changed or not). This is especially tiresome when indention is changed. 2. It makes it more difficult to track changes in the history. 3. Other users do not necessarily agree with your (or your IDE's) formatting and will format it back. So it might get back and forth. There is nothing wrong with adding a space between arguments here and there, but please keep reformatting changes to a minimum.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106520755

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +116,58 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregationStateType: RowTypeInfo =
          — End diff –

          this is not used

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106520755 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = — End diff – this is not used
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106517261

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +116,58 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + private[flink] def CreateBoundedProcessingOverGlobalWindowFunction(
          — End diff –

          indent -2

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106517261 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + private [flink] def CreateBoundedProcessingOverGlobalWindowFunction( — End diff – indent -2
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106532760

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala —
          @@ -0,0 +1,311 @@
          +/*
          + * 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.api.scala.stream.sql
          +
          +import org.apache.flink.api.scala._
          +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
          +import org.apache.flink.table.api.

          { TableEnvironment, TableException }

          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.api.scala.stream.utils.

          { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +}

          +import org.junit.Assert._
          +import org.junit._
          +import scala.collection.mutable
          +import org.apache.flink.types.Row
          +
          +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase {
          — End diff –

          This class is an integration tests because it starts a Flink minicluster. Hence its name should end with `ITCase` and not `Test` which would indicate a lightweight unit test.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106532760 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala — @@ -0,0 +1,311 @@ +/* + * 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.api.scala.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api. { TableEnvironment, TableException } +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils. { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +} +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable +import org.apache.flink.types.Row + +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase { — End diff – This class is an integration tests because it starts a Flink minicluster. Hence its name should end with `ITCase` and not `Test` which would indicate a lightweight unit test.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106538072

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -106,9 +113,14 @@ class DataStreamOverAggregate(
          if (overWindow.lowerBound.isUnbounded &&
          overWindow.upperBound.isCurrentRow)

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]
          — End diff –

          What is the check `overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]` for and why do you check if `overWindow.upperBound.isPreceding`?

          Should the condition rather be like this:
          ```
          else if (overWindow.lowerBound.isPreceding() && !overWindow.lowerBound.isUnbounded() && // bounded preceding
          overWindow.upperBound.isCurrentRow() && // until current row
          overWindow.isRows) // is rows window
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106538072 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,9 +113,14 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] — End diff – What is the check `overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] ` for and why do you check if `overWindow.upperBound.isPreceding`? Should the condition rather be like this: ``` else if (overWindow.lowerBound.isPreceding() && !overWindow.lowerBound.isUnbounded() && // bounded preceding overWindow.upperBound.isCurrentRow() && // until current row overWindow.isRows) // is rows window ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106523715

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +116,58 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + private[flink] def CreateBoundedProcessingOverGlobalWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): AllWindowFunction[Row,Row,GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverAllWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + def getLowerBoundary(
          + constants: ImmutableList[RexLiteral],
          + lowerBound: RexWindowBound,
          + input: RelNode):Int = {
          + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf[RexInputRef]
          + val index:Int = ref.getIndex
          + val count: Int = input.getRowType.getFieldCount
          + val lowerBoundIndex = count - index;
          — End diff –

          Please remove `;`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106523715 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + private [flink] def CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): AllWindowFunction [Row,Row,GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverAllWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode):Int = { + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf [RexInputRef] + val index:Int = ref.getIndex + val count: Int = input.getRowType.getFieldCount + val lowerBoundIndex = count - index; — End diff – Please remove `;`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106534604

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala —
          @@ -0,0 +1,311 @@
          +/*
          + * 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.api.scala.stream.sql
          +
          +import org.apache.flink.api.scala._
          +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
          +import org.apache.flink.table.api.

          { TableEnvironment, TableException }

          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.api.scala.stream.utils.

          { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +}

          +import org.junit.Assert._
          +import org.junit._
          +import scala.collection.mutable
          +import org.apache.flink.types.Row
          +
          +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase {
          — End diff –

          Each ITCase class adds significantly to the build time of the project because it starts a mini cluster which takes some time. Therefore, we try to keep the number of ITCase classes low and aim to test as much as possible with faster unit tests.

          Please add unit test methods that test the translation of queries to `org.apache.flink.table.api.scala.stream.sql.WindowAggregateTest`.

          Please add the following integration tests to `org.apache.flink.table.api.scala.stream.sql.SqlITCase`

          • multiple aggregation functions with partitioning
          • multiple aggregation functions without partitioning

          These tests classes contain already a few test methods that you can use as a starting point.

          Thank you

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106534604 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala — @@ -0,0 +1,311 @@ +/* + * 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.api.scala.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api. { TableEnvironment, TableException } +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils. { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +} +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable +import org.apache.flink.types.Row + +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase { — End diff – Each ITCase class adds significantly to the build time of the project because it starts a mini cluster which takes some time. Therefore, we try to keep the number of ITCase classes low and aim to test as much as possible with faster unit tests. Please add unit test methods that test the translation of queries to `org.apache.flink.table.api.scala.stream.sql.WindowAggregateTest`. Please add the following integration tests to `org.apache.flink.table.api.scala.stream.sql.SqlITCase` multiple aggregation functions with partitioning multiple aggregation functions without partitioning These tests classes contain already a few test methods that you can use as a starting point. Thank you
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106517060

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -89,6 +116,58 @@ object AggregateUtil

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          — End diff –

          Please align the arguments as the other functions do.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106517060 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], — End diff – Please align the arguments as the other functions do.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106519460

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,77 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput()) + 1
            +
            + val result: DataStream[Row] =
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty)

            { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            // global non-partitioned aggregation
            + else {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction(
            + namedAggregates,
            + inputType)
            +
            + inputDS
            + .countWindowAll(lowerbound,1)
            + .apply(windowFunction)
            + .setParallelism(1).setMaxParallelism(1)

              • End diff –

          Parallelism 1 is already implied by `countWindowAll`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106519460 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,77 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + 1 + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates, + inputType) + + inputDS + .countWindowAll(lowerbound,1) + .apply(windowFunction) + .setParallelism(1).setMaxParallelism(1) End diff – Parallelism 1 is already implied by `countWindowAll`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106528174

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * 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 org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
          +
          +class BoundedProcessingOverAllWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          + private var reuse: Row = _
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + if (null == accumulators)

          { + accumulators = new Row(aggregates.length) + }

          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + // setting the accumulators for each aggregation
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          accumulators can be reset with `aggregate.resetAccumulator(acc)`. So we can initialize once and reuse them

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106528174 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * 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 org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction + +class BoundedProcessingOverAllWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulators: Row = _ + private var reuse: Row = _ + + output = new Row(forwardedFieldCount + aggregates.length) + if (null == accumulators) { + accumulators = new Row(aggregates.length) + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + // setting the accumulators for each aggregation + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – accumulators can be reset with `aggregate.resetAccumulator(acc)`. So we can initialize once and reuse them
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106516596

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -17,35 +17,62 @@
          */
          package org.apache.flink.table.runtime.aggregate

          -import java.util
          +import scala.collection.JavaConversions.asScalaBuffer
          — End diff –

          Can we keep the imports as they are?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106516596 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -17,35 +17,62 @@ */ package org.apache.flink.table.runtime.aggregate -import java.util +import scala.collection.JavaConversions.asScalaBuffer — End diff – Can we keep the imports as they are?