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

Add event time OVER ROWS BETWEEN UNBOUNDED PRECEDING aggregation to SQL

    Details

    • Type: Sub-task
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • 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 RANGE aggregations on event time streams to the SQL interface.

      Queries similar to the following should be supported:

      SELECT 
        a, 
        SUM(b) OVER (PARTITION BY c ORDER BY rowTime() ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS sumB,
        MIN(b) OVER (PARTITION BY c ORDER BY rowTime() ROWS BETWEEN UNBOUNDED 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 rowTime() as parameter. rowTime() is a parameterless scalar function that just indicates processing time mode.
      • bounded PRECEDING is not supported (see FLINK-5655)
      • FOLLOWING is not supported.

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

      An event-time OVER ROWS window will not be able to handle late data, because this would mean in insert a row into a sorted order shift all other computations. This would be too expensive to maintain. Therefore, we will throw an error if a user tries to use an event-time OVER ROWS window with late data handling.

      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
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r108352888

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          Updated JIRA FLINK-5658(https://issues.apache.org/jira/browse/FLINK-5658). @sunjincheng121, not sure if that was an issue with your permissions. It is not possible to modify a closed issue. You need to reopen it first.

          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/3386#discussion_r108352888 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – Updated JIRA FLINK-5658 ( https://issues.apache.org/jira/browse/FLINK-5658 ). @sunjincheng121, not sure if that was an issue with your permissions. It is not possible to modify a closed issue. You need to reopen it first.
          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/3386#discussion_r108352300

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          Oh, I just recalled why I hadn't added an event-time OVER ROWS windows. They cannot deal with late data because all results would need to be shifted if a late record would be inserted.

          I would propose to keep this implementation anyway, and just throw an exception if a user enables handling of late data. What do you think @hongyuhong and @sunjincheng121?

          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/3386#discussion_r108352300 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – Oh, I just recalled why I hadn't added an event-time OVER ROWS windows. They cannot deal with late data because all results would need to be shifted if a late record would be inserted. I would propose to keep this implementation anyway, and just throw an exception if a user enables handling of late data. What do you think @hongyuhong and @sunjincheng121?
          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/3386#discussion_r108314395

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          Hi @fhueske I had created a new JIRA. FLINK-6200(https://issues.apache.org/jira/browse/FLINK-6200) for the `RANGE` case. But I do not have permission to modify this JIRA's name. So, can you help me to do it?
          Hi, @hongyuhong I'm glad to hear that you want to continue do the `RANGE` case. Feel free you want to take FLINK-6200(https://issues.apache.org/jira/browse/FLINK-6200).

          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/3386#discussion_r108314395 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – Hi @fhueske I had created a new JIRA. FLINK-6200 ( https://issues.apache.org/jira/browse/FLINK-6200 ) for the `RANGE` case. But I do not have permission to modify this JIRA's name. So, can you help me to do it? Hi, @hongyuhong I'm glad to hear that you want to continue do the `RANGE` case. Feel free you want to take FLINK-6200 ( https://issues.apache.org/jira/browse/FLINK-6200 ).
          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/3386#discussion_r108030107

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          Yes, renaming the JIRA and a new for the `RANGE` case would be great! 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/3386#discussion_r108030107 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – Yes, renaming the JIRA and a new for the `RANGE` case would be great! Thanks
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r108026230

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          Hi @sunjincheng121,thanks for your reminding, and i am glad to supplement it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r108026230 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – Hi @sunjincheng121,thanks for your reminding, and i am glad to supplement it.
          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/3386#discussion_r108025485

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          No problem. So,Can I change this JIRA.'s title, And open a new JIRA. to addresses the `RANGE` case ? @fhueske
          Thanks,
          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/3386#discussion_r108025485 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – No problem. So,Can I change this JIRA.'s title, And open a new JIRA. to addresses the `RANGE` case ? @fhueske Thanks, SunJincheng
          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/3386#discussion_r107987832

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          I see, so this PR addresses the `ROW` case.
          I'll push out a hotfix.

          Thanks for the notification @sunjincheng121!

          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/3386#discussion_r107987832 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – I see, so this PR addresses the `ROW` case. I'll push out a hotfix. Thanks for the notification @sunjincheng121!
          Hide
          fhueske Fabian Hueske added a comment -

          Implemented with fe2c61a28e6a5300b2cf4c1e50ee884b51ef42c9

          Show
          fhueske Fabian Hueske added a comment - Implemented with fe2c61a28e6a5300b2cf4c1e50ee884b51ef42c9
          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/3386#discussion_r107982106

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,12 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
            + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
              • End diff –

          EVENT-TIME OVER Need to treatment "rows" and "range" clause separately, because

          • ROWS specifies the window in physical units (rows).
          • RANGE specifies the window as a logical offset.

          They have different semantics, for example:
          DATA:
          ```
          (long, int, String)
          (1L, 1, "H")
          (2L, 2, "H")
          (2L, 3,"H")
          ```
          ROWS sum(b) result: `1,3,6`
          RANGE sum(b) result: `1,6,6`

          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/3386#discussion_r107982106 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,12 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + createUnboundedAndCurrentRowEventTimeOverWindow(inputDS) End diff – EVENT-TIME OVER Need to treatment "rows" and "range" clause separately, because ROWS specifies the window in physical units (rows). RANGE specifies the window as a logical offset. They have different semantics, for example: DATA: ``` (long, int, String) (1L, 1, "H") (2L, 2, "H") (2L, 3,"H") ``` ROWS sum(b) result: `1,3,6` RANGE sum(b) result: `1,6,6`
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

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

          Github user fhueske commented on the issue:

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

          Thanks for the update @hongyuhong!

          I will run some final tests and merge this PR

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3386 Thanks for the update @hongyuhong! I will run some final tests and merge this PR
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hongyuhong commented on the issue:

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

          Thanks, @fhueske, i have fixed it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on the issue: https://github.com/apache/flink/pull/3386 Thanks, @fhueske, i have fixed it.
          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/3386#discussion_r107719825

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,200 @@
          +/*
          + * 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, LinkedList, List => JList}

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

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.java.typeutils.

          {ListTypeInfo}

          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var sortList: LinkedList[Long] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
          + val collector = out.asInstanceOf[TimestampedCollector[Row]]
          +
          + val mapIter = rowMapState.iterator
          + if (mapIter.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + var i = 0
          +
          + // sort record according timestamp
          + do {
          + val entry = mapIter.next
          + if (entry.getKey <= curWatermark) {
          — End diff –

          we could set an `earlyRows` flag to `true` and avoid to get an iterator on the Map state to figure out that we have rows left.

          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/3386#discussion_r107719825 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,200 @@ +/* + * 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, LinkedList, List => JList} + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.java.typeutils. {ListTypeInfo} +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var sortList: LinkedList [Long] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector [Row] ]) + val collector = out.asInstanceOf[TimestampedCollector [Row] ] + + val mapIter = rowMapState.iterator + if (mapIter.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + var i = 0 + + // sort record according timestamp + do { + val entry = mapIter.next + if (entry.getKey <= curWatermark) { — End diff – we could set an `earlyRows` flag to `true` and avoid to get an iterator on the Map state to figure out that we have rows left.
          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/3386#discussion_r107713264

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,200 @@
          +/*
          + * 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, LinkedList, List => JList}

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

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.java.typeutils.

          {ListTypeInfo}

          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var sortList: LinkedList[Long] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
          + val collector = out.asInstanceOf[TimestampedCollector[Row]]
          +
          + val mapIter = rowMapState.iterator
          + if (mapIter.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + var i = 0
          +
          + // sort record according timestamp
          — End diff –

          -> `sort timestamp keys`

          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/3386#discussion_r107713264 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,200 @@ +/* + * 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, LinkedList, List => JList} + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.java.typeutils. {ListTypeInfo} +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var sortList: LinkedList [Long] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector [Row] ]) + val collector = out.asInstanceOf[TimestampedCollector [Row] ] + + val mapIter = rowMapState.iterator + if (mapIter.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + var i = 0 + + // sort record according timestamp — End diff – -> `sort timestamp keys`
          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/3386#discussion_r107712202

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,200 @@
          +/*
          + * 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, LinkedList, List => JList}

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

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.java.typeutils.

          {ListTypeInfo}

          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var sortList: LinkedList[Long] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) {
          + // ensure every key just register on timer
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          +
          + val rowList =
          + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp)
          — End diff –

          we can directly call `get()` and check for `null`. This will save one call to the state backend.

          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/3386#discussion_r107712202 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,200 @@ +/* + * 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, LinkedList, List => JList} + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.java.typeutils. {ListTypeInfo} +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var sortList: LinkedList [Long] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) — End diff – we can directly call `get()` and check for `null`. This will save one call to the state backend.
          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/3386#discussion_r107712950

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,200 @@
          +/*
          + * 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, LinkedList, List => JList}

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

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.java.typeutils.

          {ListTypeInfo}

          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var sortList: LinkedList[Long] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
          + val collector = out.asInstanceOf[TimestampedCollector[Row]]
          +
          + val mapIter = rowMapState.iterator
          — End diff –

          we should use `rowMapState.keys` to only retrieve the keys.
          If we iterate over the entries, also the values will be deserialized.

          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/3386#discussion_r107712950 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,200 @@ +/* + * 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, LinkedList, List => JList} + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.java.typeutils. {ListTypeInfo} +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var sortList: LinkedList [Long] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector [Row] ]) + val collector = out.asInstanceOf[TimestampedCollector [Row] ] + + val mapIter = rowMapState.iterator — End diff – we should use `rowMapState.keys` to only retrieve the keys. If we iterate over the entries, also the values will be deserialized.
          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/3386#discussion_r107714536

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,200 @@
          +/*
          + * 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, LinkedList, List => JList}

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

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.java.typeutils.

          {ListTypeInfo}

          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowMapState: MapState[Long, JList[Row]] = _
          + private var sortList: LinkedList[Long] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
          + val collector = out.asInstanceOf[TimestampedCollector[Row]]
          +
          + val mapIter = rowMapState.iterator
          + if (mapIter.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + var i = 0
          +
          + // sort record according timestamp
          + do {
          + val entry = mapIter.next
          + if (entry.getKey <= curWatermark)

          { + insertToSortedList(entry.getKey) + }

          + } while (mapIter.hasNext)
          +
          + // get last accumulator
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + while (i < aggregates.length)

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

          + }
          +
          + // emit the output in order
          + while (!sortList.isEmpty) {
          + val curTimeStamp = sortList.removeFirst()
          + val curRowList = rowMapState.get(curTimeStamp)
          + collector.setAbsoluteTimestamp(curTimeStamp)
          +
          + var j = 0
          + while (j < curRowList.size) {
          + val curRow = curRowList.get(j)
          + i = 0
          + while (i < forwardedFieldCount)

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

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

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

          + collector.collect(output)
          + j += 1
          + }
          + rowMapState.remove(curTimeStamp)
          + }
          +
          + accumulatorState.update(lastAccumulator)
          +
          + // if still exist records not emit this time, register a timer
          + if (rowMapState.iterator.hasNext) {
          — End diff –

          check the `earlyRows` flag to avoid another state access?

          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/3386#discussion_r107714536 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,200 @@ +/* + * 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, LinkedList, List => JList} + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.java.typeutils. {ListTypeInfo} +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowMapState: MapState[Long, JList [Row] ] = _ + private var sortList: LinkedList [Long] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + sortList = new LinkedList[Long]() + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val rowListTypeInfo: TypeInformation[JList[Row]] = + new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]] + val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("rowmapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) + rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + val rowList = + if (rowMapState.contains(ctx.timestamp)) rowMapState.get(ctx.timestamp) + else new ArrayList[Row]() + rowList.add(input) + rowMapState.put(ctx.timestamp, rowList) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector [Row] ]) + val collector = out.asInstanceOf[TimestampedCollector [Row] ] + + val mapIter = rowMapState.iterator + if (mapIter.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + var i = 0 + + // sort record according timestamp + do { + val entry = mapIter.next + if (entry.getKey <= curWatermark) { + insertToSortedList(entry.getKey) + } + } while (mapIter.hasNext) + + // get last accumulator + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + while (i < aggregates.length) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + // emit the output in order + while (!sortList.isEmpty) { + val curTimeStamp = sortList.removeFirst() + val curRowList = rowMapState.get(curTimeStamp) + collector.setAbsoluteTimestamp(curTimeStamp) + + var j = 0 + while (j < curRowList.size) { + val curRow = curRowList.get(j) + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, curRow.getField(i)) + i += 1 + } + + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curRow.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + collector.collect(output) + j += 1 + } + rowMapState.remove(curTimeStamp) + } + + accumulatorState.update(lastAccumulator) + + // if still exist records not emit this time, register a timer + if (rowMapState.iterator.hasNext) { — End diff – check the `earlyRows` flag to avoid another state access?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hongyuhong commented on the issue:

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

          Hi, @fhueske , i have updated the pr. Thanks very much.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on the issue: https://github.com/apache/flink/pull/3386 Hi, @fhueske , i have updated the pr. Thanks very much.
          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/3386#discussion_r107611456

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          — End diff –

          Hi @hongyuhong, have a look at the [comment](https://github.com/apache/flink/pull/3574#issuecomment-288646109) I made on PR #3574.
          If we use `MapState` here, we only need to sort and store the keys (longs) and can access the values as needed. This would reduce the memory requirements.

          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/3386#discussion_r107611456 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() — End diff – Hi @hongyuhong, have a look at the [comment] ( https://github.com/apache/flink/pull/3574#issuecomment-288646109 ) I made on PR #3574. If we use `MapState` here, we only need to sort and store the keys (longs) and can access the values as needed. This would reduce the memory requirements.
          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/3386#discussion_r107609421

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "max(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "min(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row) " +
          + "from T1"
          +
          + val t1 = env.addSource[(Int, Long, String)](new SourceFunction[(Int, Long, String)] {
          + override def run(ctx: SourceContext[(Int, Long, String)]): Unit =

          { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + }

          +
          + override def cancel(): Unit = {}
          + }).toTable(tEnv).as('a, 'b, 'c)
          +
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + val expected = mutable.MutableList(
          + "1,2,Hello,2,1,2,2,2",
          + "1,3,Hello world,5,2,2,3,2",
          + "1,1,Hi,6,3,2,3,1",
          + "2,1,Hello,1,1,1,1,1",
          + "2,2,Hello world,3,2,1,2,1",
          + "3,1,Hello,1,1,1,1,1",
          + "3,2,Hello world,3,2,1,2,1",
          + "1,5,Hello world,11,4,2,5,1",
          + "1,6,Hello world,17,5,3,6,1",
          + "1,9,Hello world,26,6,4,9,1",
          + "1,8,Hello world,34,7,4,9,1",
          + "1,7,Hello world,41,8,5,9,1",
          + "2,5,Hello world,8,3,2,5,1",
          + "3,5,Hello world,8,3,2,5,1"
          + )
          + assertEquals(expected.sorted, StreamITCase.testResults.sorted)
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          — End diff –

          I think you are right. If we have a parallelism of > 1, the watermarks might advance later, so we would have less rows discarded as late data. This makes the test non deterministic.

          I think what we can do is to implement a test without late data and increase the parallelism. That should work, IMO.

          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/3386#discussion_r107609421 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,193 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "max(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "min(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row) " + + "from T1" + + val t1 = env.addSource [(Int, Long, String)] (new SourceFunction [(Int, Long, String)] { + override def run(ctx: SourceContext [(Int, Long, String)] ): Unit = { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + } + + override def cancel(): Unit = {} + }).toTable(tEnv).as('a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "1,2,Hello,2,1,2,2,2", + "1,3,Hello world,5,2,2,3,2", + "1,1,Hi,6,3,2,3,1", + "2,1,Hello,1,1,1,1,1", + "2,2,Hello world,3,2,1,2,1", + "3,1,Hello,1,1,1,1,1", + "3,2,Hello world,3,2,1,2,1", + "1,5,Hello world,11,4,2,5,1", + "1,6,Hello world,17,5,3,6,1", + "1,9,Hello world,26,6,4,9,1", + "1,8,Hello world,34,7,4,9,1", + "1,7,Hello world,41,8,5,9,1", + "2,5,Hello world,8,3,2,5,1", + "3,5,Hello world,8,3,2,5,1" + ) + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) — End diff – I think you are right. If we have a parallelism of > 1, the watermarks might advance later, so we would have less rows discarded as late data. This makes the test non deterministic. I think what we can do is to implement a test without late data and increase the parallelism. That should work, IMO.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r107597005

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          — End diff –

          As my understanding, even we use RocksDBState, when we use MapState.get or ListState.get, it will malloc enough memory to store the deserialize value, this part memory is the same to sortList, then how should we control the memory of this part? Cause we need some structure to help us do the sort action, currently i can not think some better state to do this.
          If you have any better solution, i'm very appreciate to know that.
          Thanks very much.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r107597005 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() — End diff – As my understanding, even we use RocksDBState, when we use MapState.get or ListState.get, it will malloc enough memory to store the deserialize value, this part memory is the same to sortList, then how should we control the memory of this part? Cause we need some structure to help us do the sort action, currently i can not think some better state to do this. If you have any better solution, i'm very appreciate to know that. Thanks very much.
          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/3386#discussion_r107585318

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          — End diff –

          IMO. we can remove both "nextWatermarkList" and "sortList". I do not support the use of data size uncontrollable memory data structures. Because in the production environment is a terrible choice,Because there are millions of window data in our application scenario. What do you think? @hongyuhong @fhueske

          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/3386#discussion_r107585318 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() — End diff – IMO. we can remove both "nextWatermarkList" and "sortList". I do not support the use of data size uncontrollable memory data structures. Because in the production environment is a terrible choice,Because there are millions of window data in our application scenario. What do you think? @hongyuhong @fhueske
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r107584162

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          — End diff –

          This reply is response to the mapState, en,, i don't know why i can't reply at that comment.
          If use MapState, we should deserialize and serialize the list everytime, i don't know if it cost less than the list add operator(RocksDB merge). And will the map.iterator loop as the order we put it in the map?

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r107584162 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 — End diff – This reply is response to the mapState, en,, i don't know why i can't reply at that comment. If use MapState, we should deserialize and serialize the list everytime, i don't know if it cost less than the list add operator(RocksDB merge). And will the map.iterator loop as the order we put it in the map?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r107583714

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "max(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "min(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row) " +
          + "from T1"
          +
          + val t1 = env.addSource[(Int, Long, String)](new SourceFunction[(Int, Long, String)] {
          + override def run(ctx: SourceContext[(Int, Long, String)]): Unit =

          { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + }

          +
          + override def cancel(): Unit = {}
          + }).toTable(tEnv).as('a, 'b, 'c)
          +
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + val expected = mutable.MutableList(
          + "1,2,Hello,2,1,2,2,2",
          + "1,3,Hello world,5,2,2,3,2",
          + "1,1,Hi,6,3,2,3,1",
          + "2,1,Hello,1,1,1,1,1",
          + "2,2,Hello world,3,2,1,2,1",
          + "3,1,Hello,1,1,1,1,1",
          + "3,2,Hello world,3,2,1,2,1",
          + "1,5,Hello world,11,4,2,5,1",
          + "1,6,Hello world,17,5,3,6,1",
          + "1,9,Hello world,26,6,4,9,1",
          + "1,8,Hello world,34,7,4,9,1",
          + "1,7,Hello world,41,8,5,9,1",
          + "2,5,Hello world,8,3,2,5,1",
          + "3,5,Hello world,8,3,2,5,1"
          + )
          + assertEquals(expected.sorted, StreamITCase.testResults.sorted)
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          — End diff –

          Hi @fhueske, i think if we just set the source of parallelism to 1, it can not work, cause DataStreamScan or DataStreamCalc will do source.map transformation, after the transformation, the parallelism will not be 1, and the data will not arrive as the order we expect, thus we cannot expect the result, what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r107583714 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,193 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "max(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "min(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row) " + + "from T1" + + val t1 = env.addSource [(Int, Long, String)] (new SourceFunction [(Int, Long, String)] { + override def run(ctx: SourceContext [(Int, Long, String)] ): Unit = { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + } + + override def cancel(): Unit = {} + }).toTable(tEnv).as('a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "1,2,Hello,2,1,2,2,2", + "1,3,Hello world,5,2,2,3,2", + "1,1,Hi,6,3,2,3,1", + "2,1,Hello,1,1,1,1,1", + "2,2,Hello world,3,2,1,2,1", + "3,1,Hello,1,1,1,1,1", + "3,2,Hello world,3,2,1,2,1", + "1,5,Hello world,11,4,2,5,1", + "1,6,Hello world,17,5,3,6,1", + "1,9,Hello world,26,6,4,9,1", + "1,8,Hello world,34,7,4,9,1", + "1,7,Hello world,41,8,5,9,1", + "2,5,Hello world,8,3,2,5,1", + "3,5,Hello world,8,3,2,5,1" + ) + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) — End diff – Hi @fhueske, i think if we just set the source of parallelism to 1, it can not work, cause DataStreamScan or DataStreamCalc will do source.map transformation, after the transformation, the parallelism will not be 1, and the data will not arrive as the order we expect, thus we cannot expect the result, 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/3386#discussion_r107385159

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "max(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "min(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row) " +
          + "from T1"
          +
          + val t1 = env.addSource[(Int, Long, String)](new SourceFunction[(Int, Long, String)] {
          + override def run(ctx: SourceContext[(Int, Long, String)]): Unit =

          { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + }

          +
          + override def cancel(): Unit = {}
          + }).toTable(tEnv).as('a, 'b, 'c)
          +
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + val expected = mutable.MutableList(
          + "1,2,Hello,2,1,2,2,2",
          + "1,3,Hello world,5,2,2,3,2",
          + "1,1,Hi,6,3,2,3,1",
          + "2,1,Hello,1,1,1,1,1",
          + "2,2,Hello world,3,2,1,2,1",
          + "3,1,Hello,1,1,1,1,1",
          + "3,2,Hello world,3,2,1,2,1",
          + "1,5,Hello world,11,4,2,5,1",
          + "1,6,Hello world,17,5,3,6,1",
          + "1,9,Hello world,26,6,4,9,1",
          + "1,8,Hello world,34,7,4,9,1",
          + "1,7,Hello world,41,8,5,9,1",
          + "2,5,Hello world,8,3,2,5,1",
          + "3,5,Hello world,8,3,2,5,1"
          + )
          + assertEquals(expected.sorted, StreamITCase.testResults.sorted)
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          — End diff –

          The test should also work correctly if we only set the parallelism of the source to 1.

          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/3386#discussion_r107385159 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,193 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "max(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "min(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row) " + + "from T1" + + val t1 = env.addSource [(Int, Long, String)] (new SourceFunction [(Int, Long, String)] { + override def run(ctx: SourceContext [(Int, Long, String)] ): Unit = { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + } + + override def cancel(): Unit = {} + }).toTable(tEnv).as('a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "1,2,Hello,2,1,2,2,2", + "1,3,Hello world,5,2,2,3,2", + "1,1,Hi,6,3,2,3,1", + "2,1,Hello,1,1,1,1,1", + "2,2,Hello world,3,2,1,2,1", + "3,1,Hello,1,1,1,1,1", + "3,2,Hello world,3,2,1,2,1", + "1,5,Hello world,11,4,2,5,1", + "1,6,Hello world,17,5,3,6,1", + "1,9,Hello world,26,6,4,9,1", + "1,8,Hello world,34,7,4,9,1", + "1,7,Hello world,41,8,5,9,1", + "2,5,Hello world,8,3,2,5,1", + "3,5,Hello world,8,3,2,5,1" + ) + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) — End diff – The test should also work correctly if we only set the parallelism of the source to 1.
          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/3386#discussion_r107388109

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          — End diff –

          can we make both lists member vals of the `ProcessFunction` 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/3386#discussion_r107388109 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() — End diff – can we make both lists member vals of the `ProcessFunction` 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/3386#discussion_r107385110

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          — End diff –

          The test should also work correctly if we only set the parallelism of the source to 1.

          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/3386#discussion_r107385110 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,193 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) — End diff – The test should also work correctly if we only set the parallelism of the source to 1.
          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/3386#discussion_r107385173

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "max(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "min(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row) " +
          + "from T1"
          +
          + val t1 = env.addSource[(Int, Long, String)](new SourceFunction[(Int, Long, String)] {
          + override def run(ctx: SourceContext[(Int, Long, String)]): Unit =

          { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + }

          +
          + override def cancel(): Unit = {}
          + }).toTable(tEnv).as('a, 'b, 'c)
          +
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + val expected = mutable.MutableList(
          + "1,2,Hello,2,1,2,2,2",
          + "1,3,Hello world,5,2,2,3,2",
          + "1,1,Hi,6,3,2,3,1",
          + "2,1,Hello,1,1,1,1,1",
          + "2,2,Hello world,3,2,1,2,1",
          + "3,1,Hello,1,1,1,1,1",
          + "3,2,Hello world,3,2,1,2,1",
          + "1,5,Hello world,11,4,2,5,1",
          + "1,6,Hello world,17,5,3,6,1",
          + "1,9,Hello world,26,6,4,9,1",
          + "1,8,Hello world,34,7,4,9,1",
          + "1,7,Hello world,41,8,5,9,1",
          + "2,5,Hello world,8,3,2,5,1",
          + "3,5,Hello world,8,3,2,5,1"
          + )
          + assertEquals(expected.sorted, StreamITCase.testResults.sorted)
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
          + "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
          + "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
          + "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
          + "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
          + "from T1"
          +
          + val t1 = env.addSource[(Int, Long, String)](new SourceFunction[(Int, Long, String)] {
          + override def run(ctx: SourceContext[(Int, Long, String)]): Unit =

          { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 2L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 5L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 3L, "Hello"), 14000003L) + ctx.collectWithTimestamp((3, 7L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((4, 9L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((5, 8L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((6, 8L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(15000020L)) + ctx.collectWithTimestamp((6, 8L, "Hello world"), 15000021L) + ctx.emitWatermark(new Watermark(15000030L)) + }

          +
          + override def cancel(): Unit = {}
          + }).toTable(tEnv).as('a, 'b, 'c)
          +
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + val expected = mutable.MutableList(
          + "2,2,Hello,2,1,2,2,2",
          + "3,5,Hello,7,2,3,5,2",
          + "1,3,Hello,10,3,3,5,2",
          + "3,7,Hello world,17,4,4,7,2",
          + "1,1,Hi,18,5,3,7,1",
          + "4,9,Hello world,27,6,4,9,1",
          + "5,8,Hello world,35,7,5,9,1",
          + "6,8,Hello world,43,8,5,9,1")
          + assertEquals(expected, StreamITCase.testResults)
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by and arrive early **/
          + @Test
          + def testUnboundedEventTimeRowWindowArriveEarly(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          — End diff –

          The test should also work correctly if we only set the parallelism of the source to 1.

          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/3386#discussion_r107385173 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,193 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "max(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "min(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row) " + + "from T1" + + val t1 = env.addSource [(Int, Long, String)] (new SourceFunction [(Int, Long, String)] { + override def run(ctx: SourceContext [(Int, Long, String)] ): Unit = { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((2, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((3, 3L, "Hello world"), 14000008L) + ctx.collectWithTimestamp((1, 5L, "Hello world"), 14000012L) + ctx.emitWatermark(new Watermark(14000020L)) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000021L) + ctx.collectWithTimestamp((1, 6L, "Hello world"), 14000019L) + ctx.collectWithTimestamp((2, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((3, 4L, "Hello world"), 14000018L) + ctx.collectWithTimestamp((2, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((3, 5L, "Hello world"), 14000022L) + ctx.collectWithTimestamp((1, 7L, "Hello world"), 14000024L) + ctx.collectWithTimestamp((1, 8L, "Hello world"), 14000023L) + ctx.collectWithTimestamp((1, 9L, "Hello world"), 14000021L) + ctx.emitWatermark(new Watermark(14000030L)) + } + + override def cancel(): Unit = {} + }).toTable(tEnv).as('a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "1,2,Hello,2,1,2,2,2", + "1,3,Hello world,5,2,2,3,2", + "1,1,Hi,6,3,2,3,1", + "2,1,Hello,1,1,1,1,1", + "2,2,Hello world,3,2,1,2,1", + "3,1,Hello,1,1,1,1,1", + "3,2,Hello world,3,2,1,2,1", + "1,5,Hello world,11,4,2,5,1", + "1,6,Hello world,17,5,3,6,1", + "1,9,Hello world,26,6,4,9,1", + "1,8,Hello world,34,7,4,9,1", + "1,7,Hello world,41,8,5,9,1", + "2,5,Hello world,8,3,2,5,1", + "3,5,Hello world,8,3,2,5,1" + ) + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " + + "count(b) over (order by rowtime() range between unbounded preceding and current row), " + + "avg(b) over (order by rowtime() range between unbounded preceding and current row), " + + "max(b) over (order by rowtime() range between unbounded preceding and current row), " + + "min(b) over (order by rowtime() range between unbounded preceding and current row) " + + "from T1" + + val t1 = env.addSource [(Int, Long, String)] (new SourceFunction [(Int, Long, String)] { + override def run(ctx: SourceContext [(Int, Long, String)] ): Unit = { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 2L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 5L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 3L, "Hello"), 14000003L) + ctx.collectWithTimestamp((3, 7L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((4, 9L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((5, 8L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((6, 8L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(15000020L)) + ctx.collectWithTimestamp((6, 8L, "Hello world"), 15000021L) + ctx.emitWatermark(new Watermark(15000030L)) + } + + override def cancel(): Unit = {} + }).toTable(tEnv).as('a, 'b, 'c) + + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "2,2,Hello,2,1,2,2,2", + "3,5,Hello,7,2,3,5,2", + "1,3,Hello,10,3,3,5,2", + "3,7,Hello world,17,4,4,7,2", + "1,1,Hi,18,5,3,7,1", + "4,9,Hello world,27,6,4,9,1", + "5,8,Hello world,35,7,5,9,1", + "6,8,Hello world,43,8,5,9,1") + assertEquals(expected, StreamITCase.testResults) + } + + /** test sliding event-time unbounded window without partitiion by and arrive early **/ + @Test + def testUnboundedEventTimeRowWindowArriveEarly(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) — End diff – The test should also work correctly if we only set the parallelism of the source to 1.
          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/3386#discussion_r107389572

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + val nextWatermarkList = new util.ArrayList[Tuple2[Long, Row]]()
          + var i = 0
          +
          + // sort record according timestamp
          + do {
          + val row = rowList.next
          + if (row.f0 > curWatermark)

          { + nextWatermarkList.add(row) + }

          else

          { + insertToSortedList(row, sortList) + }

          + } while (rowList.hasNext)
          +
          + // emit the output in order
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + while (i < aggregates.length)

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

          + }
          +
          + val listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + i = 0
          + while (i < forwardedFieldCount)

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

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

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

          +
          + out match {
          — End diff –

          can we move this match out of the loop at the beginning of `onTimer()`?

          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/3386#discussion_r107389572 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + val nextWatermarkList = new util.ArrayList[Tuple2 [Long, Row] ]() + var i = 0 + + // sort record according timestamp + do { + val row = rowList.next + if (row.f0 > curWatermark) { + nextWatermarkList.add(row) + } else { + insertToSortedList(row, sortList) + } + } while (rowList.hasNext) + + // emit the output in order + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + while (i < aggregates.length) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + i += 1 + } + + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + + out match { — End diff – can we move this match out of the loop at the beginning of `onTimer()`?
          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/3386#discussion_r107390469

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + val nextWatermarkList = new util.ArrayList[Tuple2[Long, Row]]()
          + var i = 0
          +
          + // sort record according timestamp
          + do {
          + val row = rowList.next
          + if (row.f0 > curWatermark)

          { + nextWatermarkList.add(row) + }

          else

          { + insertToSortedList(row, sortList) + }

          + } while (rowList.hasNext)
          +
          + // emit the output in order
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + while (i < aggregates.length)

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

          + }
          +
          + val listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + i = 0
          + while (i < forwardedFieldCount)

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

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

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

          +
          + out match

          { + case collect: TimestampedCollector[Row] => collect.setAbsoluteTimestamp(curTuple.f0) + case _ => + }

          + out.collect(output)
          + }
          +
          + accumulatorState.update(lastAccumulator)
          + rowState.clear
          — End diff –

          add `()` because method modifies state

          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/3386#discussion_r107390469 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + val nextWatermarkList = new util.ArrayList[Tuple2 [Long, Row] ]() + var i = 0 + + // sort record according timestamp + do { + val row = rowList.next + if (row.f0 > curWatermark) { + nextWatermarkList.add(row) + } else { + insertToSortedList(row, sortList) + } + } while (rowList.hasNext) + + // emit the output in order + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + while (i < aggregates.length) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + i += 1 + } + + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + + out match { + case collect: TimestampedCollector[Row] => collect.setAbsoluteTimestamp(curTuple.f0) + case _ => + } + out.collect(output) + } + + accumulatorState.update(lastAccumulator) + rowState.clear — End diff – add `()` because method modifies state
          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/3386#discussion_r107387246

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          — End diff –

          Can we change this to `import org.apache.flink.api.java.tuple.

          {Tuple2 => JTuple2}

          ` to indicate that this is not a Scala Tuple2?

          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/3386#discussion_r107387246 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 — End diff – Can we change this to `import org.apache.flink.api.java.tuple. {Tuple2 => JTuple2} ` to indicate that this is not a Scala Tuple2?
          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/3386#discussion_r107386829

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          — End diff –

          @hongyuhong, what do you think about using `MapState[Long, List[Row]]` here instead of `ListState[Tuple2[Long, 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/3386#discussion_r107386829 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ — End diff – @hongyuhong, what do you think about using `MapState[Long, List [Row] ]` here instead of `ListState[Tuple2 [Long, 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/3386#discussion_r107383711

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -91,6 +91,33 @@ object AggregateUtil {
          }

          /**
          + * Create an [[ProcessFunction]] to evaluate final aggregate value.
          + *
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @return [[UnboundedEventTimeOverProcessFunction]]
          + */
          + private[flink] def CreateUnboundedEventTimeOverProcessFunction(
          — End diff –

          methods should start with lowercase -> `createUnboundedEventTimeOverProcessFunction`

          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/3386#discussion_r107383711 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -91,6 +91,33 @@ object AggregateUtil { } /** + * Create an [ [ProcessFunction] ] to evaluate final aggregate value. + * + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @return [ [UnboundedEventTimeOverProcessFunction] ] + */ + private [flink] def CreateUnboundedEventTimeOverProcessFunction( — End diff – methods should start with lowercase -> `createUnboundedEventTimeOverProcessFunction`
          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/3386#discussion_r107392253

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "max(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "min(b) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row) " +
          + "from T1"
          +
          + val t1 = env.addSource[(Int, Long, String)](new SourceFunction[(Int, Long, String)] {
          + override def run(ctx: SourceContext[(Int, Long, String)]): Unit = {
          + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L)
          + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L)
          + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L)
          + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L)
          + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L)
          + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L)
          + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L)
          + ctx.emitWatermark(new Watermark(14000010L))
          + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L)
          — End diff –

          indicate which records are late and will be dropped.

          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/3386#discussion_r107392253 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,193 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "max(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "min(b) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row) " + + "from T1" + + val t1 = env.addSource [(Int, Long, String)] (new SourceFunction [(Int, Long, String)] { + override def run(ctx: SourceContext [(Int, Long, String)] ): Unit = { + ctx.collectWithTimestamp((1, 1L, "Hi"), 14000005L) + ctx.collectWithTimestamp((2, 1L, "Hello"), 14000000L) + ctx.collectWithTimestamp((3, 1L, "Hello"), 14000002L) + ctx.collectWithTimestamp((1, 2L, "Hello"), 14000003L) + ctx.collectWithTimestamp((1, 3L, "Hello world"), 14000004L) + ctx.collectWithTimestamp((3, 2L, "Hello world"), 14000007L) + ctx.collectWithTimestamp((2, 2L, "Hello world"), 14000008L) + ctx.emitWatermark(new Watermark(14000010L)) + ctx.collectWithTimestamp((1, 4L, "Hello world"), 14000008L) — End diff – indicate which records are late and will be dropped.
          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/3386#discussion_r107387271

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          — End diff –

          this import is unused.

          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/3386#discussion_r107387271 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer — End diff – this import is unused.
          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/3386#discussion_r107388771

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,201 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param input The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + val rowList = rowState.get.iterator
          + if (rowList.hasNext) {
          + val curWatermark = ctx.timerService.currentWatermark
          + val sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + val nextWatermarkList = new util.ArrayList[Tuple2[Long, Row]]()
          + var i = 0
          +
          + // sort record according timestamp
          + do {
          + val row = rowList.next
          + if (row.f0 > curWatermark)

          { + nextWatermarkList.add(row) + }

          else

          { + insertToSortedList(row, sortList) + }

          + } while (rowList.hasNext)
          +
          + // emit the output in order
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + while (i < aggregates.length)

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

          + }
          +
          + val listIter = sortList.listIterator()
          — End diff –

          Can we use `removeFirst()` instead of creating an iterator?
          So basically:

          ```
          while (!sortList.isEmpty)

          { val curTuple = sortList.removeFirst() ... }

          ```

          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/3386#discussion_r107388771 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,201 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", intermediateType) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tuple2Type: TypeInformation[Tuple2[Long, Row]] = + new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType) + .asInstanceOf[TypeInformation[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tuple2Type) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param input The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + val rowList = rowState.get.iterator + if (rowList.hasNext) { + val curWatermark = ctx.timerService.currentWatermark + val sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + val nextWatermarkList = new util.ArrayList[Tuple2 [Long, Row] ]() + var i = 0 + + // sort record according timestamp + do { + val row = rowList.next + if (row.f0 > curWatermark) { + nextWatermarkList.add(row) + } else { + insertToSortedList(row, sortList) + } + } while (rowList.hasNext) + + // emit the output in order + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + while (i < aggregates.length) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + } + + val listIter = sortList.listIterator() — End diff – Can we use `removeFirst()` instead of creating an iterator? So basically: ``` while (!sortList.isEmpty) { val curTuple = sortList.removeFirst() ... } ```
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hongyuhong commented on the issue:

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

          Hi @fhueske , @shijinkui & @sunjincheng121 , i have updated the pr according to your advices. Thanks very much.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on the issue: https://github.com/apache/flink/pull/3386 Hi @fhueske , @shijinkui & @sunjincheng121 , i have updated the pr according to your advices. Thanks very much.
          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/3386#discussion_r107115163

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          — End diff –

          value -> input

          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/3386#discussion_r107115163 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. — End diff – value -> input
          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/3386#discussion_r107102856

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          — End diff –

          Hi @sunjincheng121, I think `ListState` is a valid approach because we sort the data before we apply aggregations and emit results. The sorting works on the assumption that the data is almost sorted, which I think is a better approach than using a general purpose data structure. Hence, out-of-order data is handled correctly and only data which is late with respect to a watermark is dropped.

          But you are right, using `MapState[Long, Row]` should be even better, because the RocksDB state backend (which is used for all serious use cases) sorts Maps based on the key (the `Long` timestamp in our case). However, we cannot assume that RocksDB is used and need to manually sort in addition. The approach of @hongyuhong would be better than a heapsort (using `PriorityQueue`) for RocksDB state backends because the data would already be sorted and we would just append to the linked list but not traverse it.

          I do not want to add an `allowedLateness` parameter yet. If we add this, it should be used for all time based windows. I'd rather drop late data and have a consistent behavior of all operators.

          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/3386#discussion_r107102856 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ — End diff – Hi @sunjincheng121, I think `ListState` is a valid approach because we sort the data before we apply aggregations and emit results. The sorting works on the assumption that the data is almost sorted, which I think is a better approach than using a general purpose data structure. Hence, out-of-order data is handled correctly and only data which is late with respect to a watermark is dropped. But you are right, using `MapState [Long, Row] ` should be even better, because the RocksDB state backend (which is used for all serious use cases) sorts Maps based on the key (the `Long` timestamp in our case). However, we cannot assume that RocksDB is used and need to manually sort in addition. The approach of @hongyuhong would be better than a heapsort (using `PriorityQueue`) for RocksDB state backends because the data would already be sorted and we would just append to the linked list but not traverse it. I do not want to add an `allowedLateness` parameter yet. If we add this, it should be used for all time based windows. I'd rather drop late data and have a consistent behavior of all operators.
          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/3386#discussion_r107062397

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          — End diff –

          I think `ListState` can not work well for event-time case. because we must deal with out of order datas,for example:
          If we allowedLateness = 2 ( the length of time that the user configures the allowable data delay)
          InputData:
          ```
          (1L, 1, "Hello"),
          (2L, 2, "Hello"),
          *(4L, 4, "Hello"),* // We should handle `4L` and `3L` elements correctly,because
          *(3L, 3, "Hello"),* //`allowedLateness=2`
          (7L, 7, "Hello"),
          (7L, 8, "Hello"),
          (5L, 5, "Hello"),
          (8L, 8, "Hello World"),
          *(20L, 20, "Hello World"),*
          *(9L, 9, "Hello World"))* // we can ignore `9L`, Because 20L-9L = 11L > 2
          ```
          So, I suggest that we can use `MapState[Long, List[Row]] ` and `PriorityQueue[(Long, Long)]` to deal with this case. then we should consider two things:
          1. Out of order but not late event.
          2. add `allowedLateness` config which use can definition.
          What do you think? @hongyuhong @fhueske

          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/3386#discussion_r107062397 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ — End diff – I think `ListState` can not work well for event-time case. because we must deal with out of order datas,for example: If we allowedLateness = 2 ( the length of time that the user configures the allowable data delay) InputData: ``` (1L, 1, "Hello"), (2L, 2, "Hello"), * (4L, 4, "Hello"), * // We should handle `4L` and `3L` elements correctly,because * (3L, 3, "Hello"), * //`allowedLateness=2` (7L, 7, "Hello"), (7L, 8, "Hello"), (5L, 5, "Hello"), (8L, 8, "Hello World"), * (20L, 20, "Hello World"), * * (9L, 9, "Hello World")) * // we can ignore `9L`, Because 20L-9L = 11L > 2 ``` So, I suggest that we can use `MapState[Long, List [Row] ] ` and `PriorityQueue [(Long, Long)] ` to deal with this case. then we should consider two things: 1. Out of order but not late event. 2. add `allowedLateness` config which use can definition. What do you think? @hongyuhong @fhueske
          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/3386#discussion_r106950646

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          — End diff –

          emit row if `curTuple.f0` > watermark else put it in a new list which is later put back into the state.

          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/3386#discussion_r106950646 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next — End diff – emit row if `curTuple.f0` > watermark else put it in a new list which is later put back into the state.
          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/3386#discussion_r106950142

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount)

          { + output.setField(i, curTuple.f1.getField(i)) + }

          +
          + for (i <- aggregates.indices)

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

          +
          + out.asInstanceOf[TimestampedCollector[Row]].setAbsoluteTimestamp(curTuple.f0)
          + out.collect(output)
          + }
          +
          + accumulatorState.update(lastAccumulator)
          + rowState.clear
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          — End diff –

          As you suggested, we need to keep elements with timestamp > watermark and register a new timer if the `rowState` is not empty.

          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/3386#discussion_r106950142 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + } + + for (i <- aggregates.indices) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + } + + out.asInstanceOf[TimestampedCollector [Row] ].setAbsoluteTimestamp(curTuple.f0) + out.collect(output) + } + + accumulatorState.update(lastAccumulator) + rowState.clear + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) — End diff – As you suggested, we need to keep elements with timestamp > watermark and register a new timer if the `rowState` is not empty.
          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/3386#discussion_r106949663

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) {
          + // ensure every key just register on timer
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          — End diff –

          Hi @hongyuhong, I completely agree with you. Very good points! 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/3386#discussion_r106949663 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) — End diff – Hi @hongyuhong, I completely agree with you. Very good points! Thanks
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106774581

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) {
          + // ensure every key just register on timer
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          — End diff –

          I think if we register use ctx.timestamp, then it will generate too much timer, if use currentWatermark + 1, then it will remove the duplicate timer, guarantee that one key will have only one timer,.
          And consider the situation like follow:
          row1: time(12)
          row2: time(14)
          row3: time(13)
          watermark:13
          ...
          watermark:20

          if we need the row2 to output until watermark:20, then i think we should put row2 back to liststate after clear. Cause even we register a timer for row2 use ctx.timestamp, and when it trigger at watermark:20, the record has already been deleted.
          So i want to register an currentwarter + 1 timer when processElement, and register a timer again in onTimer when still have rows.
          What do you think? @fhueske

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106774581 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) — End diff – I think if we register use ctx.timestamp, then it will generate too much timer, if use currentWatermark + 1, then it will remove the duplicate timer, guarantee that one key will have only one timer,. And consider the situation like follow: row1: time(12) row2: time(14) row3: time(13) watermark:13 ... watermark:20 if we need the row2 to output until watermark:20, then i think we should put row2 back to liststate after clear. Cause even we register a timer for row2 use ctx.timestamp, and when it trigger at watermark:20, the record has already been deleted. So i want to register an currentwarter + 1 timer when processElement, and register a timer again in onTimer when still have rows. What do you think? @fhueske
          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/3386#discussion_r106622821

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          — End diff –

          check immediately if `rowList.hasNext()` and leave the function if `rowList` is empty. This function will be called very often with empty lists so we should keep the overhead as low as possible.

          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/3386#discussion_r106622821 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator — End diff – check immediately if `rowList.hasNext()` and leave the function if `rowList` is empty. This function will be called very often with empty lists so we should keep the overhead as low as possible.
          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/3386#discussion_r106623012

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          — End diff –

          this should have been checked before. So we can skip the check.

          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/3386#discussion_r106623012 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { — End diff – this should have been checked before. So we can skip the check.
          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/3386#discussion_r106617971

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -112,7 +113,14 @@ class DataStreamOverAggregate(
          "condition.")
          }
          case _: RowTimeType =>

          • throw new TableException("OVER Window of the EventTime type is not currently supported.")
            + if (overWindow.lowerBound.isUnbounded &&
            + overWindow.upperBound.isCurrentRow) {
              • End diff –

          move this condition into the line 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/3386#discussion_r106617971 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -112,7 +113,14 @@ class DataStreamOverAggregate( "condition.") } case _: RowTimeType => throw new TableException("OVER Window of the EventTime type is not currently supported.") + if (overWindow.lowerBound.isUnbounded && + overWindow.upperBound.isCurrentRow) { End diff – move this condition into the line 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/3386#discussion_r106623827

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount)

          { + output.setField(i, curTuple.f1.getField(i)) + }

          +
          + for (i <- aggregates.indices) {
          — End diff –

          `while` loop

          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/3386#discussion_r106623827 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + } + + for (i <- aggregates.indices) { — End diff – `while` loop
          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/3386#discussion_r106620612

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

          + def createUnboundedAndCurrentRowEventTimeOverWindow(
          + 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]
          +
          + val result: DataStream[Row] =
          + // partitioned aggregation
          + if (partitionKeys.nonEmpty)

          { + val keyedStream = inputDS.keyBy(partitionKeys: _*) + val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction( + namedAggregates, + inputType) + + keyedStream + .process(processFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

          + // global non-partitioned aggregation
          + else {
          + val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction(
          + namedAggregates,
          + inputType)
          +
          + inputDS.keyBy(new NullByteKeySelector[Row])
          + .process(processFunction)
          + .setParallelism(1)
          — End diff –

          also `setMaxParallelism(1)` to prevent that this operator can be scaled out.

          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/3386#discussion_r106620612 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -159,6 +167,46 @@ class DataStreamOverAggregate( result } + def createUnboundedAndCurrentRowEventTimeOverWindow( + 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] + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val keyedStream = inputDS.keyBy(partitionKeys: _*) + val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction( + namedAggregates, + inputType) + + keyedStream + .process(processFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + // global non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction( + namedAggregates, + inputType) + + inputDS.keyBy(new NullByteKeySelector [Row] ) + .process(processFunction) + .setParallelism(1) — End diff – also `setMaxParallelism(1)` to prevent that this operator can be scaled out.
          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/3386#discussion_r106623704

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount) {
          — End diff –

          `while` loop

          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/3386#discussion_r106623704 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { — End diff – `while` loop
          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/3386#discussion_r106621564

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -91,6 +91,35 @@ object AggregateUtil {
          }

          /**
          + * Create an [[ProcessFunction]] to evaluate final aggregate value.
          + *
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @return [[UnboundedProcessingOverProcessFunction]]
          + */
          + private[flink] def CreateUnboundedEventTimeOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): UnboundedEventTimeOverProcessFunction = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregationStateType: RowTypeInfo =
          — End diff –

          Also you can use `createAccumulatorRowType(inputType, aggregates)`.

          Btw. could you refactor the `createAccumulatorRowType(inputType, aggregates)` method and remove the `inputType` parameter? It is not used. 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/3386#discussion_r106621564 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -91,6 +91,35 @@ object AggregateUtil { } /** + * Create an [ [ProcessFunction] ] to evaluate final aggregate value. + * + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @return [ [UnboundedProcessingOverProcessFunction] ] + */ + private [flink] def CreateUnboundedEventTimeOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): UnboundedEventTimeOverProcessFunction = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = — End diff – Also you can use `createAccumulatorRowType(inputType, aggregates)`. Btw. could you refactor the `createAccumulatorRowType(inputType, aggregates)` method and remove the `inputType` parameter? It is not used. 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/3386#discussion_r106624514

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount)

          { + output.setField(i, curTuple.f1.getField(i)) + }

          +
          + for (i <- aggregates.indices)

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

          +
          + out.asInstanceOf[TimestampedCollector[Row]].setAbsoluteTimestamp(curTuple.f0)
          + out.collect(output)
          + }
          +
          + accumulatorState.update(lastAccumulator)
          + rowState.clear
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          — End diff –

          Why do you register another timer? Don't we need a timer only if new data was added 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/3386#discussion_r106624514 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + } + + for (i <- aggregates.indices) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + } + + out.asInstanceOf[TimestampedCollector [Row] ].setAbsoluteTimestamp(curTuple.f0) + out.collect(output) + } + + accumulatorState.update(lastAccumulator) + rowState.clear + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) — End diff – Why do you register another timer? Don't we need a timer only if new data was added 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/3386#discussion_r106622479

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) {
          + // ensure every key just register on timer
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          — End diff –

          we should register the timer based on the record timestamp:
          `ctx.timerService.registerEventTimeTimer(ctx.timestamp + 1)`

          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/3386#discussion_r106622479 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) — End diff – we should register the timer based on the record timestamp: `ctx.timerService.registerEventTimeTimer(ctx.timestamp + 1)`
          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/3386#discussion_r106625475

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(a) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(a) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(a) over (" +
          — End diff –

          Computing `avg`, `max`, `min` on the partition key is not very meaningful. Can you compute those on `b`?

          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/3386#discussion_r106625475 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,119 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(a) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(a) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(a) over (" + — End diff – Computing `avg`, `max`, `min` on the partition key is not very meaningful. Can you compute those on `b`?
          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/3386#discussion_r106626434

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          — End diff –

          Can you also add a few unit tests to `org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala` to verify that the query is correctly translated? 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/3386#discussion_r106626434 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,119 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { — End diff – Can you also add a few unit tests to `org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala` to verify that the query is correctly translated? 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/3386#discussion_r106623601

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices) {
          — End diff –

          use a `while` loop. `for` loops are not very efficient in Scala.
          ```
          var i = 0
          while (i < aggregates.length)

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

          ```

          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/3386#discussion_r106623601 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { — End diff – use a `while` loop. `for` loops are not very efficient in Scala. ``` var i = 0 while (i < aggregates.length) { lastAccumulator.setField(i, aggregates(i).createAccumulator()) i += 1 } ```
          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/3386#discussion_r106624582

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,181 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + /**
          + * Process one element from the input stream, not emit the output
          + *
          + * @param value The input value.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + *
          + */
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + /**
          + * Called when a timer set fires, sort current records according the timestamp
          + * and emit the output
          + *
          + * @param timestamp The timestamp of the firing timer.
          + * @param ctx The ctx to register timer or get current time
          + * @param out The collector for returning result values.
          + */
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          +
          + // sort record according timestamp
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + // emit the output in order
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount)

          { + output.setField(i, curTuple.f1.getField(i)) + }

          +
          + for (i <- aggregates.indices)

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

          +
          + out.asInstanceOf[TimestampedCollector[Row]].setAbsoluteTimestamp(curTuple.f0)
          + out.collect(output)
          + }
          +
          + accumulatorState.update(lastAccumulator)
          + rowState.clear
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          +
          + }
          +
          + }
          +
          + /**
          + * consider disorder records are in the minority,so reverse searche location
          — End diff –

          `searche` -> `search`

          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/3386#discussion_r106624582 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,181 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + /** + * Process one element from the input stream, not emit the output + * + * @param value The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + /** + * Called when a timer set fires, sort current records according the timestamp + * and emit the output + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + + // sort record according timestamp + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + // emit the output in order + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + } + + for (i <- aggregates.indices) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + } + + out.asInstanceOf[TimestampedCollector [Row] ].setAbsoluteTimestamp(curTuple.f0) + out.collect(output) + } + + accumulatorState.update(lastAccumulator) + rowState.clear + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + } + + } + + /** + * consider disorder records are in the minority,so reverse searche location — End diff – `searche` -> `search`
          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/3386#discussion_r106625858

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

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

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setStateBackend(getStateBackend)
          + StreamITCase.testResults = mutable.MutableList()
          + env.setParallelism(1)
          +
          + val sqlQuery = "SELECT a, b, c, " +
          + "SUM(a) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "count(a) over (" +
          + "partition by a order by rowtime() range between unbounded preceding and current row), " +
          + "avg(a) over (" +
          — End diff –

          Also, most groups have just a single record. The max is two records. With that we cannot really check if the sorting works correctly. Can you make less groups (less distinct `a` values) and add more rows for some groups with out-of-order timestamps?

          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/3386#discussion_r106625858 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +320,119 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testUnboundedEventTimeRowWindowWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setStateBackend(getStateBackend) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + + val sqlQuery = "SELECT a, b, c, " + + "SUM(a) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "count(a) over (" + + "partition by a order by rowtime() range between unbounded preceding and current row), " + + "avg(a) over (" + — End diff – Also, most groups have just a single record. The max is two records. With that we cannot really check if the sorting works correctly. Can you make less groups (less distinct `a` values) and add more rows for some groups with out-of-order timestamps?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106592513

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,159 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount)

          { + output.setField(i, curTuple.f1.getField(i)) + }

          +
          + for (i <- aggregates.indices)

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

          +
          + out.asInstanceOf[TimestampedCollector[Row]].setAbsoluteTimestamp(curTuple.f0)
          + out.collect(output)
          + }
          +
          + accumulatorState.update(lastAccumulator)
          + rowState.clear
          + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1)
          +
          — End diff –

          no new line

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106592513 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,159 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + } + + for (i <- aggregates.indices) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + } + + out.asInstanceOf[TimestampedCollector [Row] ].setAbsoluteTimestamp(curTuple.f0) + out.collect(output) + } + + accumulatorState.update(lastAccumulator) + rowState.clear + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + — End diff – no new line
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106590822

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,159 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          — End diff –

          `val listIter = sortList.listIterator()`

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106590822 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,159 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() — End diff – `val listIter = sortList.listIterator()`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106590692

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,159 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + while (rowList.hasNext) {
          — End diff –

          `rowList` and `sortList` use `val` to declare if needn't re-assign a new value below.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106590692 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,159 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + while (rowList.hasNext) { — End diff – `rowList` and `sortList` use `val` to declare if needn't re-assign a new value below.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106592195

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,159 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark())

          { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + }

          + }
          +
          + override def onTimer(
          + timestamp: Long,
          + ctx: ProcessFunction[Row, Row]#OnTimerContext,
          + out: Collector[Row]): Unit = {
          +
          + var rowList = rowState.get.iterator
          + var sortList = new util.LinkedList[Tuple2[Long, Row]]()
          + while (rowList.hasNext)

          { + insertToSortedList(rowList.next, sortList) + }

          +
          + if (!sortList.isEmpty) {
          + var lastAccumulator = accumulatorState.value
          + if (lastAccumulator == null) {
          + lastAccumulator = new Row(aggregates.length)
          + for (i <- aggregates.indices)

          { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + }

          + }
          +
          + var listIter = sortList.listIterator()
          + while (listIter.hasNext) {
          + val curTuple = listIter.next
          + for (i <- 0 until forwardedFieldCount)

          { + output.setField(i, curTuple.f1.getField(i)) + }

          +
          + for (i <- aggregates.indices)

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

          +
          + out.asInstanceOf[TimestampedCollector[Row]].setAbsoluteTimestamp(curTuple.f0)
          — End diff –

          The function parameter is `out: Collector[Row]`, so here we don't know whether its type is `TimestampedCollector[Row]]`.
          use match/case to check the `TimestampedCollector[Row]]` type and then `setAbsoluteTimestamp`. This will be safe.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106592195 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,159 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( + getRuntimeContext.getExecutionConfig).asInstanceOf[TypeSerializer[Tuple2[Long, Row]]] + val tupleStateDescriptor: ListStateDescriptor[Tuple2[Long, Row]] = + new ListStateDescriptor[Tuple2[Long, Row]]("rowliststate", tupleSerializer) + rowState = getRuntimeContext.getListState[Tuple2[Long, Row]](tupleStateDescriptor) + + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + // ensure every key just register on timer + ctx.timerService.registerEventTimeTimer(ctx.timerService.currentWatermark + 1) + + rowState.add(new Tuple2(ctx.timestamp, input)) + } + } + + override def onTimer( + timestamp: Long, + ctx: ProcessFunction [Row, Row] #OnTimerContext, + out: Collector [Row] ): Unit = { + + var rowList = rowState.get.iterator + var sortList = new util.LinkedList[Tuple2 [Long, Row] ]() + while (rowList.hasNext) { + insertToSortedList(rowList.next, sortList) + } + + if (!sortList.isEmpty) { + var lastAccumulator = accumulatorState.value + if (lastAccumulator == null) { + lastAccumulator = new Row(aggregates.length) + for (i <- aggregates.indices) { + lastAccumulator.setField(i, aggregates(i).createAccumulator()) + } + } + + var listIter = sortList.listIterator() + while (listIter.hasNext) { + val curTuple = listIter.next + for (i <- 0 until forwardedFieldCount) { + output.setField(i, curTuple.f1.getField(i)) + } + + for (i <- aggregates.indices) { + val index = forwardedFieldCount + i + val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, curTuple.f1.getField(aggFields(i))) + output.setField(index, aggregates(i).getValue(accumulator)) + } + + out.asInstanceOf[TimestampedCollector [Row] ].setAbsoluteTimestamp(curTuple.f0) — End diff – The function parameter is `out: Collector [Row] `, so here we don't know whether its type is `TimestampedCollector [Row] ]`. use match/case to check the `TimestampedCollector [Row] ]` type and then `setAbsoluteTimestamp`. This will be safe.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106590304

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -91,6 +91,35 @@ object AggregateUtil {
          }

          /**
          + * Create an [[ProcessFunction]] to evaluate final aggregate value.
          + *
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @return [[UnboundedProcessingOverProcessFunction]]
          + */
          + private[flink] def CreateUnboundedEventTimeOverProcessFunction(
          — End diff –

          function name should start with lowercase

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106590304 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -91,6 +91,35 @@ object AggregateUtil { } /** + * Create an [ [ProcessFunction] ] to evaluate final aggregate value. + * + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @return [ [UnboundedProcessingOverProcessFunction] ] + */ + private [flink] def CreateUnboundedEventTimeOverProcessFunction( — End diff – function name should start with lowercase
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106590214

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

          + def createUnboundedAndCurrentRowEventTimeOverWindow(
          + 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]
          +
          + val result: DataStream[Row] =
          + // partitioned aggregation
          + if (partitionKeys.nonEmpty) {
          + val keyedStream = inputDS.keyBy(partitionKeys: _*)
          + val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction(
          — End diff –

          `val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction(` can declared before the if/else, because it have no any relationship with `partitionKeys`

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106590214 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -159,6 +167,46 @@ class DataStreamOverAggregate( result } + def createUnboundedAndCurrentRowEventTimeOverWindow( + 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] + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val keyedStream = inputDS.keyBy(partitionKeys: _*) + val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction( — End diff – `val processFunction = AggregateUtil.CreateUnboundedEventTimeOverProcessFunction(` can declared before the if/else, because it have no any relationship with `partitionKeys`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106589672

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala —
          @@ -0,0 +1,159 @@
          +/*
          + * 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
          +
          +import org.apache.flink.api.common.typeinfo.

          {BasicTypeInfo, TypeInformation}

          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.java.tuple.Tuple2
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param intermediateType the intermediate row tye which the state saved
          + * @param inputType the input row tye which the state saved
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val intermediateType: TypeInformation[Row],
          + private val inputType: TypeInformation[Row])
          + extends ProcessFunction[Row, Row]{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulatorState: ValueState[Row] = _
          + private var rowState: ListState[Tuple2[Long, Row]] = _
          +
          +
          + override def open(config: Configuration) {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + val valueSerializer: TypeSerializer[Row] =
          + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig)
          + val stateDescriptor: ValueStateDescriptor[Row] =
          + new ValueStateDescriptor[Row]("accumulatorstate", valueSerializer)
          + accumulatorState = getRuntimeContext.getState[Row](stateDescriptor)
          +
          + val tupleSerializer: TypeSerializer[Tuple2[Long, Row]] =
          + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer(
          — End diff –

          `(new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType))` can omit the outside brackets.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106589672 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,159 @@ +/* + * 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 + +import org.apache.flink.api.common.typeinfo. {BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.tuple.Tuple2 +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param intermediateType the intermediate row tye which the state saved + * @param inputType the input row tye which the state saved + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val intermediateType: TypeInformation [Row] , + private val inputType: TypeInformation [Row] ) + extends ProcessFunction [Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulatorState: ValueState [Row] = _ + private var rowState: ListState[Tuple2 [Long, Row] ] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer [Row] = + intermediateType.createSerializer(getRuntimeContext.getExecutionConfig) + val stateDescriptor: ValueStateDescriptor [Row] = + new ValueStateDescriptor [Row] ("accumulatorstate", valueSerializer) + accumulatorState = getRuntimeContext.getState [Row] (stateDescriptor) + + val tupleSerializer: TypeSerializer[Tuple2 [Long, Row] ] = + (new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType)).createSerializer( — End diff – `(new TupleTypeInfo(BasicTypeInfo.LONG_TYPE_INFO, inputType))` can omit the outside brackets.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106590402

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -91,6 +91,35 @@ object AggregateUtil {
          }

          /**
          + * Create an [[ProcessFunction]] to evaluate final aggregate value.
          + *
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @return [[UnboundedProcessingOverProcessFunction]]
          + */
          + private[flink] def CreateUnboundedEventTimeOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): UnboundedEventTimeOverProcessFunction = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregationStateType: RowTypeInfo =
          — End diff –

          ```
          val aggregationStateType: RowTypeInfo =
          createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
          ```
          This will be more readable.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106590402 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -91,6 +91,35 @@ object AggregateUtil { } /** + * Create an [ [ProcessFunction] ] to evaluate final aggregate value. + * + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @return [ [UnboundedProcessingOverProcessFunction] ] + */ + private [flink] def CreateUnboundedEventTimeOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): UnboundedEventTimeOverProcessFunction = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = — End diff – ``` val aggregationStateType: RowTypeInfo = createDataSetAggregateBufferDataType(Array(), aggregates, inputType) ``` This will be more readable.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user shijinkui commented on the issue:

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

          @hongyuhong , IMO, all the function should have detail doc

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on the issue: https://github.com/apache/flink/pull/3386 @hongyuhong , IMO, all the function should have detail doc
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hongyuhong commented on the issue:

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

          Hi @fhueske @sunjincheng121 @rtudoran @shijinkui , i have update the pr, hope you can take a look. Thanks advance.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on the issue: https://github.com/apache/flink/pull/3386 Hi @fhueske @sunjincheng121 @rtudoran @shijinkui , i have update the pr, hope you can take a look. Thanks advance.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106411203

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,138 @@
          +/*
          + * 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.TimeCharacteristic
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks
          +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
          +import org.apache.flink.streaming.api.watermark.Watermark
          +import org.apache.flink.table.api.

          {TableEnvironment, TableException}

          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.api.scala.stream.utils.StreamTestData.Small4Tuple
          +import org.apache.flink.table.api.scala.stream.utils.

          {StreamITCase, StreamTestData, StreamingWithStateTestBase}

          +import org.apache.flink.types.Row
          +import org.junit.Assert._
          +import org.junit._
          +
          +import scala.collection.mutable
          +
          +class UnboundedRowtimeOverTest extends StreamingWithStateTestBase {
          — End diff –

          @hongyuhong all the scala UT or IT file name should end with `ITCase` or `Suite`

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106411203 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,138 @@ +/* + * 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.TimeCharacteristic +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.api.watermark.Watermark +import org.apache.flink.table.api. {TableEnvironment, TableException} +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils.StreamTestData.Small4Tuple +import org.apache.flink.table.api.scala.stream.utils. {StreamITCase, StreamTestData, StreamingWithStateTestBase} +import org.apache.flink.types.Row +import org.junit.Assert._ +import org.junit._ + +import scala.collection.mutable + +class UnboundedRowtimeOverTest extends StreamingWithStateTestBase { — End diff – @hongyuhong all the scala UT or IT file name should end with `ITCase` or `Suite`
          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/3386#discussion_r106412030

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          — End diff –

          We can of course keep a branch with these changes around.

          However, I'd also like to add a non-retract version of for event-time OVER UNBOUNDED windows. We haven't started with the work on retractions yet. So it is not sure that we will make it for the 1.3 release.

          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/3386#discussion_r106412030 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = — End diff – We can of course keep a branch with these changes around. However, I'd also like to add a non-retract version of for event-time OVER UNBOUNDED windows. We haven't started with the work on retractions yet. So it is not sure that we will make it for the 1.3 release.
          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/3386#discussion_r106409879

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          — End diff –

          @fhueske - I understand - do you think we should put this in a branch and use it as a model to update the retraction for the others? It might be mixed with the design document proposed for retraction on the mailing list. What do you think?

          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/3386#discussion_r106409879 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = — End diff – @fhueske - I understand - do you think we should put this in a branch and use it as a model to update the retraction for the others? It might be mixed with the design document proposed for retraction on the mailing list. 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/3386#discussion_r106386147

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          — End diff –

          Waiting for watermarks is the only way to handle out-of-order data without sending out retractions. Since, retractions are not supported yet, it is the only viable option. Otherwise, we would need to drop all records which have not monotonically increasing timestamps.

          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/3386#discussion_r106386147 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( — End diff – Waiting for watermarks is the only way to handle out-of-order data without sending out retractions. Since, retractions are not supported yet, it is the only viable option. Otherwise, we would need to drop all records which have not monotonically increasing timestamps.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106373065

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          — End diff –

          Hi @fhueske, thanks very much for your suggestion. I still have some confuse about that. In my understanding, row window should trigger per row, i'm wonder if it's appropriate to ouput until watermark? What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106373065 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( — End diff – Hi @fhueske, thanks very much for your suggestion. I still have some confuse about that. In my understanding, row window should trigger per row, i'm wonder if it's appropriate to ouput until watermark? 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/3386#discussion_r106366395

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          — End diff –

          Adding retraction support is a much bigger issue. If we add support for this operator and merge it to master, the master branch is in an inconsistent state because some operators support it and other don't.

          We must ensure that adding retraction support is an "atomic" operation, i.e., either we add support for all operators or none. I will not be in the situation that the work is half (or 80%) done when the 1.3 release branch is forked off on May 1st.

          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/3386#discussion_r106366395 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = — End diff – Adding retraction support is a much bigger issue. If we add support for this operator and merge it to master, the master branch is in an inconsistent state because some operators support it and other don't. We must ensure that adding retraction support is an "atomic" operation, i.e., either we add support for all operators or none. I will not be in the situation that the work is half (or 80%) done when the 1.3 release branch is forked off on May 1st.
          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/3386#discussion_r106363331

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          — End diff –

          @fhueske - I am confused about this reply. In the previous discussions on the mailing list as well in the reply on the discussion about supporting retraction for Flink Streaming - the idea was to build such a feature.

          you said "IMO, we must make sure that either all operators support retraction or none" - why shouldn't we use this as a blue print on how to build the retraction for the others?

          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/3386#discussion_r106363331 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = — End diff – @fhueske - I am confused about this reply. In the previous discussions on the mailing list as well in the reply on the discussion about supporting retraction for Flink Streaming - the idea was to build such a feature. you said "IMO, we must make sure that either all operators support retraction or none" - why shouldn't we use this as a blue print on how to build the retraction for the others?
          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/3386#discussion_r106363300

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          — End diff –

          @fhueske - I am confused about this reply. In the previous discussions on the mailing list as well in the reply on the discussion about supporting retraction for Flink Streaming - the idea was to build such a feature.

          you said "IMO, we must make sure that either all operators support retraction or none" - why shouldn't we use this as a blue print on how to build the retraction for the others?

          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/3386#discussion_r106363300 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = — End diff – @fhueske - I am confused about this reply. In the previous discussions on the mailing list as well in the reply on the discussion about supporting retraction for Flink Streaming - the idea was to build such a feature. you said "IMO, we must make sure that either all operators support retraction or none" - why shouldn't we use this as a blue print on how to build the retraction for the others?
          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/3386#discussion_r106359002

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          — End diff –

          Yes, we collect all records between watermarks. When a watermark is received we compute the aggregates and emit the results.

          That's basically the price of not being able to sent out retractions.

          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/3386#discussion_r106359002 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( — End diff – Yes, we collect all records between watermarks. When a watermark is received we compute the aggregates and emit the results. That's basically the price of not being able to sent out retractions.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3386#discussion_r106324953

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          — End diff –

          Is that means we should emit the output until the watermark arrive?

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r106324953 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( — End diff – Is that means we should emit the output until the watermark arrive?
          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/3386#discussion_r106220993

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          — End diff –

          Very great design, it is very similar to bound event-time over window design, the difference is that I need to use retract method.
          +1

          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/3386#discussion_r106220993 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( — End diff – Very great design, it is very similar to bound event-time over window design, the difference is that I need to use retract method. +1
          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/3386#discussion_r106183250

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val key = keySelector.getKey(input)
          + val timeSections = if (timeSectionsMap.containsKey(key)) timeSectionsMap.get(key)
          + else new util.LinkedList[TimeWindow]()
          +
          + expire(key, ctx.timerService.currentWatermark, timeSections)
          +
          + // discard later record
          + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) {
          +
          + timeSectionsMap.put(key, timeSections)
          +
          + // find the last accumulator with the same key before current timestamp
          + // and find the accumulators need to retraction
          + val (closestTimeOption: Option[TimeWindow],
          + newTimeSection: TimeWindow,
          + retractions: Array[TimeWindow]) =
          + resolveTimeSection(ctx.timestamp,timeSections)
          +
          + val newAccumulators = new Row(forwardedFieldCount + aggregates.length)
          + aggregateWithIndex.foreach { case (agg, i) =>
          — End diff –

          Use `while` loops to iterate over the aggregates. Scala's `foreach` loops have overhead

          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/3386#discussion_r106183250 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val key = keySelector.getKey(input) + val timeSections = if (timeSectionsMap.containsKey(key)) timeSectionsMap.get(key) + else new util.LinkedList [TimeWindow] () + + expire(key, ctx.timerService.currentWatermark, timeSections) + + // discard later record + if (ctx.timestamp() >= ctx.timerService().currentWatermark()) { + + timeSectionsMap.put(key, timeSections) + + // find the last accumulator with the same key before current timestamp + // and find the accumulators need to retraction + val (closestTimeOption: Option [TimeWindow] , + newTimeSection: TimeWindow, + retractions: Array [TimeWindow] ) = + resolveTimeSection(ctx.timestamp,timeSections) + + val newAccumulators = new Row(forwardedFieldCount + aggregates.length) + aggregateWithIndex.foreach { case (agg, i) => — End diff – Use `while` loops to iterate over the aggregates. Scala's `foreach` loops have overhead
          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/3386#discussion_r106181911

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          — End diff –

          `intermediate`

          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/3386#discussion_r106181911 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , — End diff – `intermediate`
          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/3386#discussion_r106201295

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -91,6 +92,38 @@ object AggregateUtil {
          }

          /**
          + * Create an [[ProcessFunction]] to evaluate final aggregate value.
          + *
          + * @param namedAggregates List of calls to aggregate functions and their output field names
          + * @param inputType Input row type
          + * @return [[UnboundedProcessingOverProcessFunction]]
          + */
          + private[flink] def CreateUnboundedEventTimeOverProcessFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType,
          + keySelector: KeySelector[Row, Tuple],
          + keyType: TypeInformation[Tuple]): UnboundedEventTimeOverProcessFunction = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregationStateType: RowTypeInfo =
          + createDataSetAggregateBufferDataType(
          + (0 until inputType.getFieldCount).toArray, aggregates, inputType)
          +
          + new UnboundedEventTimeOverProcessFunction(
          + aggregates,
          + aggFields,
          + inputType.getFieldCount,
          + aggregationStateType,
          + keySelector,
          — End diff –

          We don't need the `keySelector` and `keyType` if we follow the design I suggest below.

          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/3386#discussion_r106201295 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -91,6 +92,38 @@ object AggregateUtil { } /** + * Create an [ [ProcessFunction] ] to evaluate final aggregate value. + * + * @param namedAggregates List of calls to aggregate functions and their output field names + * @param inputType Input row type + * @return [ [UnboundedProcessingOverProcessFunction] ] + */ + private [flink] def CreateUnboundedEventTimeOverProcessFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType, + keySelector: KeySelector [Row, Tuple] , + keyType: TypeInformation [Tuple] ): UnboundedEventTimeOverProcessFunction = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType( + (0 until inputType.getFieldCount).toArray, aggregates, inputType) + + new UnboundedEventTimeOverProcessFunction( + aggregates, + aggFields, + inputType.getFieldCount, + aggregationStateType, + keySelector, — End diff – We don't need the `keySelector` and `keyType` if we follow the design I suggest below.
          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/3386#discussion_r106203455

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/UnboundedEventTimeOverProcessFuncTest.java —
          @@ -0,0 +1,240 @@
          +/*
          + * 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
          + * <p>
          + * http://www.apache.org/licenses/LICENSE-2.0
          + * <p>
          + * 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.java.stream.sql;
          +
          +
          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.functions.KeySelector;
          +import org.apache.flink.api.java.tuple.Tuple;
          +import org.apache.flink.api.java.tuple.Tuple1;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo;
          +import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
          +import org.apache.flink.streaming.api.operators.ProcessOperator;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
          +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
          +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
          +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
          +import org.apache.flink.streaming.util.TestHarnessUtil;
          +import org.apache.flink.table.functions.AggregateFunction;
          +import org.apache.flink.table.functions.aggfunctions.IntSumAggFunction;
          +import org.apache.flink.table.runtime.aggregate.UnboundedEventTimeOverProcessFunction;
          +import org.apache.flink.types.Row;
          +import org.apache.flink.util.TestLogger;
          +import org.junit.Test;
          +
          +import java.util.concurrent.ConcurrentLinkedQueue;
          +
          +
          +/**
          + * Tests

          {@link ProcessOperator}

          .
          + */
          +public class UnboundedEventTimeOverProcessFuncTest extends TestLogger {
          — End diff –

          Please implement the test in Scala

          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/3386#discussion_r106203455 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/UnboundedEventTimeOverProcessFuncTest.java — @@ -0,0 +1,240 @@ +/* + * 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 + * <p> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p> + * 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.java.stream.sql; + + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.api.operators.ProcessOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.aggfunctions.IntSumAggFunction; +import org.apache.flink.table.runtime.aggregate.UnboundedEventTimeOverProcessFunction; +import org.apache.flink.types.Row; +import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Tests {@link ProcessOperator} . + */ +public class UnboundedEventTimeOverProcessFuncTest extends TestLogger { — End diff – Please implement the test in Scala
          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/3386#discussion_r106186862

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          — End diff –

          The current state of the Table API does not know how to handle retractions.
          So we can send every result just once and never correct it.

          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/3386#discussion_r106186862 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = — End diff – The current state of the Table API does not know how to handle retractions. So we can send every result just once and never correct it.
          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/3386#discussion_r106184455

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          — End diff –

          We use `while` loops to iterate over the AggregateFunctions. So we do not need to `zipWithIndex`.

          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/3386#discussion_r106184455 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex — End diff – We use `while` loops to iterate over the AggregateFunctions. So we do not need to `zipWithIndex`.
          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/3386#discussion_r106190096

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

          {ByteArrayInputStream, ByteArrayOutputStream}

          +import java.util
          +
          +import org.apache.flink.api.common.typeinfo.TypeInformation
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.types.Row
          +import org.apache.flink.streaming.api.functions.

          {ProcessFunction}

          +import org.apache.flink.util.

          {Collector, Preconditions}

          +import org.apache.flink.api.common.state._
          +import org.apache.flink.api.common.typeutils.TypeSerializer
          +import org.apache.flink.api.common.typeutils.base.StringSerializer
          +import org.apache.flink.api.java.functions.KeySelector
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.core.memory.

          {DataInputViewStreamWrapper, DataOutputViewStreamWrapper}

          +import org.apache.flink.runtime.state.

          {FunctionInitializationContext, FunctionSnapshotContext}

          +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
          +import org.apache.flink.streaming.api.operators.TimestampedCollector
          +import org.apache.flink.streaming.api.windowing.windows.TimeWindow
          +import org.apache.flink.table.functions.

          {Accumulator, AggregateFunction}

          +
          +import scala.collection.mutable.ArrayBuffer
          +
          +/**
          + * A ProcessFunction to support unbounded event-time over-window
          + *
          + * @param aggregates the aggregate functions
          + * @param aggFields the filed index which the aggregate functions use
          + * @param forwardedFieldCount the input fields count
          + * @param interMediateType the intermediate row tye which the state saved
          + * @param keySelector the keyselector
          + * @param keyType the key type
          + *
          + */
          +class UnboundedEventTimeOverProcessFunction(
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int,
          + private val interMediateType: TypeInformation[Row],
          + private val keySelector: KeySelector[Row, Tuple],
          + private val keyType: TypeInformation[Tuple])
          + extends ProcessFunction[Row, Row]
          + with CheckpointedFunction{
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var state: MapState[TimeWindow, Row] = _
          + private val aggregateWithIndex: Array[(AggregateFunction[_], Int)] = aggregates.zipWithIndex
          +
          + /** Sorted list per key for choose the recent result and the records need retraction **/
          + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]] =
          + new java.util.HashMap[Tuple, java.util.LinkedList[TimeWindow]]
          +
          + /** For store timeSectionsMap **/
          + private var timeSectionsState: ListState[String] = _
          + private var inputKeySerializer: TypeSerializer[Tuple] = _
          + private var timeSerializer: TypeSerializer[TimeWindow] = _
          +
          + override def open(config: Configuration)

          { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + }

          +
          + override def processElement(
          + input: Row,
          + ctx: ProcessFunction[Row, Row]#Context,
          + out: Collector[Row]): Unit = {
          +
          + val key = keySelector.getKey(input)
          — End diff –

          The key management should be left to Flink.
          Flink puts keyed state automatically in the context of the key of the currently processed record.

          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/3386#discussion_r106190096 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala — @@ -0,0 +1,283 @@ +/* + * 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.io. {ByteArrayInputStream, ByteArrayOutputStream} +import java.util + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration +import org.apache.flink.types.Row +import org.apache.flink.streaming.api.functions. {ProcessFunction} +import org.apache.flink.util. {Collector, Preconditions} +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.base.StringSerializer +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.core.memory. {DataInputViewStreamWrapper, DataOutputViewStreamWrapper} +import org.apache.flink.runtime.state. {FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.streaming.api.windowing.windows.TimeWindow +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} + +import scala.collection.mutable.ArrayBuffer + +/** + * A ProcessFunction to support unbounded event-time over-window + * + * @param aggregates the aggregate functions + * @param aggFields the filed index which the aggregate functions use + * @param forwardedFieldCount the input fields count + * @param interMediateType the intermediate row tye which the state saved + * @param keySelector the keyselector + * @param keyType the key type + * + */ +class UnboundedEventTimeOverProcessFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int, + private val interMediateType: TypeInformation [Row] , + private val keySelector: KeySelector [Row, Tuple] , + private val keyType: TypeInformation [Tuple] ) + extends ProcessFunction [Row, Row] + with CheckpointedFunction{ + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var state: MapState [TimeWindow, Row] = _ + private val aggregateWithIndex: Array[(AggregateFunction [_] , Int)] = aggregates.zipWithIndex + + /** Sorted list per key for choose the recent result and the records need retraction **/ + private val timeSectionsMap: java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] = + new java.util.HashMap[Tuple, java.util.LinkedList [TimeWindow] ] + + /** For store timeSectionsMap **/ + private var timeSectionsState: ListState [String] = _ + private var inputKeySerializer: TypeSerializer [Tuple] = _ + private var timeSerializer: TypeSerializer [TimeWindow] = _ + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + val valueSerializer: TypeSerializer[Row] = + interMediateType.createSerializer(getRuntimeContext.getExecutionConfig) + timeSerializer = new TimeWindow.Serializer + val stateDescriptor: MapStateDescriptor[TimeWindow, Row] = + new MapStateDescriptor[TimeWindow, Row]("rowtimeoverstate", timeSerializer, valueSerializer) + inputKeySerializer = keyType.createSerializer(getRuntimeContext.getExecutionConfig) + state = getRuntimeContext.getMapState[TimeWindow, Row](stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction [Row, Row] #Context, + out: Collector [Row] ): Unit = { + + val key = keySelector.getKey(input) — End diff – The key management should be left to Flink. Flink puts keyed state automatically in the context of the key of the currently processed record.
          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/3386#discussion_r106207575

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,138 @@
          +/*
          + * 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.TimeCharacteristic
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks
          +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
          +import org.apache.flink.streaming.api.watermark.Watermark
          +import org.apache.flink.table.api.

          {TableEnvironment, TableException}

          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.api.scala.stream.utils.StreamTestData.Small4Tuple
          +import org.apache.flink.table.api.scala.stream.utils.

          {StreamITCase, StreamTestData, StreamingWithStateTestBase}

          +import org.apache.flink.types.Row
          +import org.junit.Assert._
          +import org.junit._
          +
          +import scala.collection.mutable
          +
          +class UnboundedRowtimeOverTest extends StreamingWithStateTestBase {
          — End diff –

          Can you add the test methods to `org.apache.flink.table.api.scala.stream.sql.SqlITCase.scala`? Each test class that extends `StreamingWithStateTestBase` adds overhead because it starts a Flink mini cluster which takes a bit of 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/3386#discussion_r106207575 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,138 @@ +/* + * 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.TimeCharacteristic +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.api.watermark.Watermark +import org.apache.flink.table.api. {TableEnvironment, TableException} +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils.StreamTestData.Small4Tuple +import org.apache.flink.table.api.scala.stream.utils. {StreamITCase, StreamTestData, StreamingWithStateTestBase} +import org.apache.flink.types.Row +import org.junit.Assert._ +import org.junit._ + +import scala.collection.mutable + +class UnboundedRowtimeOverTest extends StreamingWithStateTestBase { — End diff – Can you add the test methods to `org.apache.flink.table.api.scala.stream.sql.SqlITCase.scala`? Each test class that extends `StreamingWithStateTestBase` adds overhead because it starts a Flink mini cluster which takes a bit of 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/3386#discussion_r106203989

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/UnboundedEventTimeOverProcessFuncTest.java —
          @@ -0,0 +1,240 @@
          +/*
          + * 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
          + * <p>
          + * http://www.apache.org/licenses/LICENSE-2.0
          + * <p>
          + * 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.java.stream.sql;
          +
          +
          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.functions.KeySelector;
          +import org.apache.flink.api.java.tuple.Tuple;
          +import org.apache.flink.api.java.tuple.Tuple1;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo;
          +import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
          +import org.apache.flink.streaming.api.operators.ProcessOperator;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
          +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
          +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
          +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
          +import org.apache.flink.streaming.util.TestHarnessUtil;
          +import org.apache.flink.table.functions.AggregateFunction;
          +import org.apache.flink.table.functions.aggfunctions.IntSumAggFunction;
          +import org.apache.flink.table.runtime.aggregate.UnboundedEventTimeOverProcessFunction;
          +import org.apache.flink.types.Row;
          +import org.apache.flink.util.TestLogger;
          +import org.junit.Test;
          +
          +import java.util.concurrent.ConcurrentLinkedQueue;
          +
          +
          +/**
          + * Tests

          {@link ProcessOperator}

          .
          + */
          +public class UnboundedEventTimeOverProcessFuncTest extends TestLogger {
          +
          + @Test
          + public void testUnboundedEventSnapshotAndRestore() throws Exception {
          +
          + AggregateFunction[] aggFunc = new AggregateFunction[1];
          + aggFunc[0] = new IntSumAggFunction();
          + int[] aggField = new int[1];
          + aggField[0] = 0;
          +
          + TypeInformation<Row> returnType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
          + TypeInformation<Row> interMediateType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, aggFunc[0].getAccumulatorType());
          — End diff –

          `interMediate` -> `intermediate`

          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/3386#discussion_r106203989 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/UnboundedEventTimeOverProcessFuncTest.java — @@ -0,0 +1,240 @@ +/* + * 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 + * <p> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p> + * 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.java.stream.sql; + + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.api.operators.ProcessOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.aggfunctions.IntSumAggFunction; +import org.apache.flink.table.runtime.aggregate.UnboundedEventTimeOverProcessFunction; +import org.apache.flink.types.Row; +import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Tests {@link ProcessOperator} . + */ +public class UnboundedEventTimeOverProcessFuncTest extends TestLogger { + + @Test + public void testUnboundedEventSnapshotAndRestore() throws Exception { + + AggregateFunction[] aggFunc = new AggregateFunction [1] ; + aggFunc [0] = new IntSumAggFunction(); + int[] aggField = new int [1] ; + aggField [0] = 0; + + TypeInformation<Row> returnType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + TypeInformation<Row> interMediateType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, aggFunc [0] .getAccumulatorType()); — End diff – `interMediate` -> `intermediate`
          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/3386#discussion_r106206858

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,138 @@
          +/*
          + * 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.TimeCharacteristic
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks
          +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
          +import org.apache.flink.streaming.api.watermark.Watermark
          +import org.apache.flink.table.api.

          {TableEnvironment, TableException}

          +import org.apache.flink.table.api.scala._
          +import org.apache.flink.table.api.scala.stream.utils.StreamTestData.Small4Tuple
          +import org.apache.flink.table.api.scala.stream.utils.

          {StreamITCase, StreamTestData, StreamingWithStateTestBase}

          +import org.apache.flink.types.Row
          +import org.junit.Assert._
          +import org.junit._
          +
          +import scala.collection.mutable
          +
          +class UnboundedRowtimeOverTest extends StreamingWithStateTestBase {
          — End diff –

          Can you add the test methods to `org.apache.flink.table.api.scala.stream.sql.SqlITCase.scala`? Each test class that extends `StreamingWithStateTestBase` adds overhead because it starts a Flink mini cluster which takes a bit of 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/3386#discussion_r106206858 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,138 @@ +/* + * 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.TimeCharacteristic +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.api.watermark.Watermark +import org.apache.flink.table.api. {TableEnvironment, TableException} +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils.StreamTestData.Small4Tuple +import org.apache.flink.table.api.scala.stream.utils. {StreamITCase, StreamTestData, StreamingWithStateTestBase} +import org.apache.flink.types.Row +import org.junit.Assert._ +import org.junit._ + +import scala.collection.mutable + +class UnboundedRowtimeOverTest extends StreamingWithStateTestBase { — End diff – Can you add the test methods to `org.apache.flink.table.api.scala.stream.sql.SqlITCase.scala`? Each test class that extends `StreamingWithStateTestBase` adds overhead because it starts a Flink mini cluster which takes a bit of 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/3386#discussion_r106203953

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/UnboundedEventTimeOverProcessFuncTest.java —
          @@ -0,0 +1,240 @@
          +/*
          + * 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
          + * <p>
          + * http://www.apache.org/licenses/LICENSE-2.0
          + * <p>
          + * 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.java.stream.sql;
          +
          +
          +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.functions.KeySelector;
          +import org.apache.flink.api.java.tuple.Tuple;
          +import org.apache.flink.api.java.tuple.Tuple1;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.api.java.typeutils.TupleTypeInfo;
          +import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
          +import org.apache.flink.streaming.api.operators.ProcessOperator;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
          +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
          +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
          +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
          +import org.apache.flink.streaming.util.TestHarnessUtil;
          +import org.apache.flink.table.functions.AggregateFunction;
          +import org.apache.flink.table.functions.aggfunctions.IntSumAggFunction;
          +import org.apache.flink.table.runtime.aggregate.UnboundedEventTimeOverProcessFunction;
          +import org.apache.flink.types.Row;
          +import org.apache.flink.util.TestLogger;
          +import org.junit.Test;
          +
          +import java.util.concurrent.ConcurrentLinkedQueue;
          +
          +
          +/**
          + * Tests

          {@link ProcessOperator}

          .
          + */
          +public class UnboundedEventTimeOverProcessFuncTest extends TestLogger {
          +
          + @Test
          + public void testUnboundedEventSnapshotAndRestore() throws Exception {
          +
          + AggregateFunction[] aggFunc = new AggregateFunction[1];
          + aggFunc[0] = new IntSumAggFunction();
          + int[] aggField = new int[1];
          + aggField[0] = 0;
          +
          + TypeInformation<Row> returnType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
          + TypeInformation<Row> interMediateType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, aggFunc[0].getAccumulatorType());
          + KeySelector<Row, Tuple> keyselector = new KeySelector<Row, Tuple>() {
          + @Override
          + public Tuple getKey(Row value) throws Exception

          { + return new Tuple1<Integer>(1); + }

          + };
          +
          + KeyedProcessOperator<Tuple, Row, Row> operator =
          + new KeyedProcessOperator<>(
          + new UnboundedEventTimeOverProcessFunction(
          + aggFunc, aggField, 1, interMediateType, keyselector,
          + new TupleTypeInfo<Tuple>(BasicTypeInfo.INT_TYPE_INFO)));
          +
          + OneInputStreamOperatorTestHarness<Row, Row> testHarness =
          + new KeyedOneInputStreamOperatorTestHarness<>(
          + operator, keyselector, new TupleTypeInfo<Tuple>(BasicTypeInfo.INT_TYPE_INFO));
          +
          + testHarness.setup();
          + testHarness.open();
          +
          + Row inputRow = new Row(1);
          + inputRow.setField(0, 1);
          + testHarness.processElement(new StreamRecord<>(inputRow, 12L));
          + testHarness.processElement(new StreamRecord<>(inputRow, 12L));
          + testHarness.processElement(new StreamRecord<>(inputRow, 12L));
          +
          + ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
          + Row res = new Row(2);
          — End diff –

          Use `Row.of()` to generate rows more concisely.

          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/3386#discussion_r106203953 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/UnboundedEventTimeOverProcessFuncTest.java — @@ -0,0 +1,240 @@ +/* + * 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 + * <p> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p> + * 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.java.stream.sql; + + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.api.operators.ProcessOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.aggfunctions.IntSumAggFunction; +import org.apache.flink.table.runtime.aggregate.UnboundedEventTimeOverProcessFunction; +import org.apache.flink.types.Row; +import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Tests {@link ProcessOperator} . + */ +public class UnboundedEventTimeOverProcessFuncTest extends TestLogger { + + @Test + public void testUnboundedEventSnapshotAndRestore() throws Exception { + + AggregateFunction[] aggFunc = new AggregateFunction [1] ; + aggFunc [0] = new IntSumAggFunction(); + int[] aggField = new int [1] ; + aggField [0] = 0; + + TypeInformation<Row> returnType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + TypeInformation<Row> interMediateType = new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, aggFunc [0] .getAccumulatorType()); + KeySelector<Row, Tuple> keyselector = new KeySelector<Row, Tuple>() { + @Override + public Tuple getKey(Row value) throws Exception { + return new Tuple1<Integer>(1); + } + }; + + KeyedProcessOperator<Tuple, Row, Row> operator = + new KeyedProcessOperator<>( + new UnboundedEventTimeOverProcessFunction( + aggFunc, aggField, 1, interMediateType, keyselector, + new TupleTypeInfo<Tuple>(BasicTypeInfo.INT_TYPE_INFO))); + + OneInputStreamOperatorTestHarness<Row, Row> testHarness = + new KeyedOneInputStreamOperatorTestHarness<>( + operator, keyselector, new TupleTypeInfo<Tuple>(BasicTypeInfo.INT_TYPE_INFO)); + + testHarness.setup(); + testHarness.open(); + + Row inputRow = new Row(1); + inputRow.setField(0, 1); + testHarness.processElement(new StreamRecord<>(inputRow, 12L)); + testHarness.processElement(new StreamRecord<>(inputRow, 12L)); + testHarness.processElement(new StreamRecord<>(inputRow, 12L)); + + ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>(); + Row res = new Row(2); — End diff – Use `Row.of()` to generate rows more concisely.