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
          Yuhong_kyo hongyuhong added a comment -

          Hi, sunjincheng & Fabian, i have some confuse about this issue.
          As processing time over unbounded window, every row will trigger an calculation and one output,
          but with event time, is that still one row has one output?
          e.g.
          A(Time:11) B(Time:12) C(Time:15) D(Time:14)
          the output is:
          A arrive: output A
          B arrive: output A reduce B
          C arrive: output A reduce B reduce C
          D arrive: output A reduce B reduce D (D is arrive later, and according to the semantics, we should calculate rows before D, so is that just calcute ABD? Thus seem to impractical cause should keep all rows)

          and if so, what's the role of the watermark?
          if no, hope can give an example to describe the expected result. Thanks.

          Show
          Yuhong_kyo hongyuhong added a comment - Hi, sunjincheng & Fabian, i have some confuse about this issue. As processing time over unbounded window, every row will trigger an calculation and one output, but with event time, is that still one row has one output? e.g. A(Time:11) B(Time:12) C(Time:15) D(Time:14) the output is: A arrive: output A B arrive: output A reduce B C arrive: output A reduce B reduce C D arrive: output A reduce B reduce D (D is arrive later, and according to the semantics, we should calculate rows before D, so is that just calcute ABD? Thus seem to impractical cause should keep all rows) and if so, what's the role of the watermark? if no, hope can give an example to describe the expected result. Thanks.
          Hide
          sunjincheng121 sunjincheng added a comment - - edited

          Hi,hongyuhong I'm glad you're interested in this jira, and I'm trying to explain your question:
          1. The semantics of OVER determines that both "processing-time" and "event-time" are output one result per row.
          2. For your example, IMO:

          • Without retraction situation, the output of the results are:
                A-> agg (A)
                B-> agg (A, B)
                C-> agg (A, B, C)
                D-> agg (A, B, D)
            
          • With retraction situation:
                A-> agg (A)
                B-> agg (A, B)
                C-> agg (A, B, C)
                D-> agg (A, B, D)
                C-> retraction agg (A, B, C)
                C-> agg (A, B, D,C)
            

            (node: implementation can be incremental)
            These are my thoughts, for reference purposes only!
            What do you think ? hongyuhong Fabian Hueske If there is any mistake please let me know.

          Show
          sunjincheng121 sunjincheng added a comment - - edited Hi, hongyuhong I'm glad you're interested in this jira, and I'm trying to explain your question: 1. The semantics of OVER determines that both "processing-time" and "event-time" are output one result per row. 2. For your example, IMO: Without retraction situation, the output of the results are: A-> agg (A) B-> agg (A, B) C-> agg (A, B, C) D-> agg (A, B, D) With retraction situation: A-> agg (A) B-> agg (A, B) C-> agg (A, B, C) D-> agg (A, B, D) C-> retraction agg (A, B, C) C-> agg (A, B, D,C) (node: implementation can be incremental) These are my thoughts, for reference purposes only! What do you think ? hongyuhong Fabian Hueske If there is any mistake please let me know.
          Hide
          fhueske Fabian Hueske added a comment -

          Hi, sunjincheng is right. In principle we need to emit a result for each arriving record.

          At the moment, the GroupWindows for the Table API simply discard late arriving records (which is the default behavior of the DataStream windows which are internally used).
          I would propose to do the same until we have the mechanics for retraction in place to be consistent with the remaining API.
          With these semantics the results would be:

              A-> agg (A)
              B-> agg (A, B)
              C-> agg (A, B, C)
              D-> _ // nothing. D would be dropped.
          

          What do you think sunjincheng and hongyuhong?

          Show
          fhueske Fabian Hueske added a comment - Hi, sunjincheng is right. In principle we need to emit a result for each arriving record. At the moment, the GroupWindows for the Table API simply discard late arriving records (which is the default behavior of the DataStream windows which are internally used). I would propose to do the same until we have the mechanics for retraction in place to be consistent with the remaining API. With these semantics the results would be: A-> agg (A) B-> agg (A, B) C-> agg (A, B, C) D-> _ // nothing. D would be dropped. What do you think sunjincheng and hongyuhong ?
          Hide
          Yuhong_kyo hongyuhong added a comment -

          HI, sunjincheng thanks for you explain. As your description, whether with retraction or without retraction, we need to keep all rows or keep all intermediate result like agg(A), agg(AB), agg(ABC), so that record arriving anytime can get the correct result, the difference is that records after D will aggregate include D or not, is that right what i understand?

          I agree with Fabian Hueske that we can do it with two phase and can have an unified retraction mechanics.

          Show
          Yuhong_kyo hongyuhong added a comment - HI, sunjincheng thanks for you explain. As your description, whether with retraction or without retraction, we need to keep all rows or keep all intermediate result like agg(A), agg(AB), agg(ABC), so that record arriving anytime can get the correct result, the difference is that records after D will aggregate include D or not, is that right what i understand? I agree with Fabian Hueske that we can do it with two phase and can have an unified retraction mechanics.
          Hide
          sunjincheng121 sunjincheng added a comment -

          HI, Fabian Hueske Agree! It's make sense at current time.
          HI, hongyuhong The answer to your question is exactly what fabian said: "D would be dropped.".

          Show
          sunjincheng121 sunjincheng added a comment - HI, Fabian Hueske Agree! It's make sense at current time. HI, hongyuhong The answer to your question is exactly what fabian said: "D would be dropped.".
          Hide
          fhueske Fabian Hueske added a comment -

          Hi hongyuhong, even with support for retraction, we would not keep the full history. The assumption is that data might arrive late, but won't be super late. So, we can continuously clean up the state after a configurable time interval behind the current event-time. This interval determines how much lateness can be accepted.

          Show
          fhueske Fabian Hueske added a comment - Hi hongyuhong , even with support for retraction, we would not keep the full history. The assumption is that data might arrive late, but won't be super late. So, we can continuously clean up the state after a configurable time interval behind the current event-time. This interval determines how much lateness can be accepted.
          Hide
          Yuhong_kyo hongyuhong added a comment -

          Got it. Thanks very much. sunjincheng i'm glad to start working on this issue immediately.

          Show
          Yuhong_kyo hongyuhong added a comment - Got it. Thanks very much. sunjincheng i'm glad to start working on this issue immediately.
          Hide
          sunjincheng121 sunjincheng added a comment -

          Sure, hongyuhong. Feel free to take it.

          Show
          sunjincheng121 sunjincheng added a comment - Sure, hongyuhong . Feel free to take it.
          Hide
          Yuhong_kyo hongyuhong added a comment -

          Hi Fabian Hueske & Haohui Mai, i saw haohui have already commit the rowtime() feature, but still no cover the LogicalWindowRelNode situation, as how to distinguish rowtime() and proctime(), i want to do like this:
          LogicalProject(with RexOver expr) -> [normalize rule(ProjectToWindow)] ->CalciteLogicalWindow(input = LogicalProject(with rowtime() func)) -> (normalize rule(FlinkLogicalWindowRule)) ->FlinkLogicalWindow(input = LogicalProject, isEventtime=true/false, window=CalciteLogicalWindow)

          Cause after normalize period, the rowtime() function will be replaced by generator code according to ReduceExpressionsRule, so we can only check whether eventtime or proctime before apply ReduceExpressionsRule in normalize period. So i want to add an FlinkLogicalWindow which include the CalciteLogicalWindow and an isEventtime attribute, and add a additional rule(FlinkLogicalWindowRule) to do the transform.

          In FlinkLogicalWindowRule, i will check according whether the function operator is EventTimeExtractor or ProcTimeExtractor.

          What do you think? or if already have solution, please let me know.

          Show
          Yuhong_kyo hongyuhong added a comment - Hi Fabian Hueske & Haohui Mai , i saw haohui have already commit the rowtime() feature, but still no cover the LogicalWindowRelNode situation, as how to distinguish rowtime() and proctime(), i want to do like this: LogicalProject(with RexOver expr) -> [normalize rule(ProjectToWindow)] ->CalciteLogicalWindow(input = LogicalProject(with rowtime() func)) -> (normalize rule(FlinkLogicalWindowRule)) ->FlinkLogicalWindow(input = LogicalProject, isEventtime=true/false, window=CalciteLogicalWindow) Cause after normalize period, the rowtime() function will be replaced by generator code according to ReduceExpressionsRule, so we can only check whether eventtime or proctime before apply ReduceExpressionsRule in normalize period. So i want to add an FlinkLogicalWindow which include the CalciteLogicalWindow and an isEventtime attribute, and add a additional rule(FlinkLogicalWindowRule) to do the transform. In FlinkLogicalWindowRule, i will check according whether the function operator is EventTimeExtractor or ProcTimeExtractor. What do you think? or if already have solution, please let me know.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user hongyuhong opened a pull request:

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

          FLINK-5658[table] support unbounded eventtime over window

          1. Add LogicalOverWindow extends calcite Window with attribute isEventtime
          2. Add LogicalWindowRule to convert calcite LogicalWindow to LogicalOverWindow, and check whether isEventtime
          3. Add DataStreamSlideEventTimeRowAgg relnode to do translateToPlan
          – partition: input.keyby().window(GlobalRowWindowAssigner).reduce
          – no partition: input.windowAll(GlobalRowWindowAssigner).reduce
          4. Add DataStreamWindowRule to convert LogicalOverWindow to DataStreamSlideEventTimeRowAgg
          5. Add GlobalEventTimeRowWindowAssigner to support eventtime and row trigger

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

          $ git pull https://github.com/hongyuhong/flink flink-5658

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

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

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

          This closes #3386


          commit 68892b76c7097b04ad7ed1e42af3162bb2e7126c
          Author: hongyuhong 00223286 <hongyuhong@huawei.com>
          Date: 2017-02-22T08:47:25Z

          flink-5658 support unbounded eventtime over window


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user hongyuhong opened a pull request: https://github.com/apache/flink/pull/3386 FLINK-5658 [table] support unbounded eventtime over window 1. Add LogicalOverWindow extends calcite Window with attribute isEventtime 2. Add LogicalWindowRule to convert calcite LogicalWindow to LogicalOverWindow, and check whether isEventtime 3. Add DataStreamSlideEventTimeRowAgg relnode to do translateToPlan – partition: input.keyby().window(GlobalRowWindowAssigner).reduce – no partition: input.windowAll(GlobalRowWindowAssigner).reduce 4. Add DataStreamWindowRule to convert LogicalOverWindow to DataStreamSlideEventTimeRowAgg 5. Add GlobalEventTimeRowWindowAssigner to support eventtime and row trigger You can merge this pull request into a Git repository by running: $ git pull https://github.com/hongyuhong/flink flink-5658 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3386.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3386 commit 68892b76c7097b04ad7ed1e42af3162bb2e7126c Author: hongyuhong 00223286 <hongyuhong@huawei.com> Date: 2017-02-22T08:47:25Z flink-5658 support unbounded eventtime over window
          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_r102630342

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode =

          { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + }

          +
          + override def toString: String = {
          + val inputFields = new Array[Int](inputType.getFieldCount)
          + for (i <- 0 until inputType.getFieldCount)
          + inputFields = i
          + s"Aggregate(${
          + if (!grouping.isEmpty) {
          + s"groupBy: ($

          {groupingToString(inputType, grouping)}

          ), "
          + } else

          { + "" + }

          + } orderBy: (eventtime), window: (unbounded), " +
          + s"select: ($

          { + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + }

          ))"
          + }
          +
          + override def explainTerms(pw: RelWriter): RelWriter =

          { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + }

          +
          + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
          +
          + val config = tableEnv.getConfig
          + val groupingKeys = grouping.toArray
          — End diff –

          Over Window should partitionKeys better.

          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_r102630342 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + } + + override def toString: String = { + val inputFields = new Array [Int] (inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields = i + s"Aggregate(${ + if (!grouping.isEmpty) { + s"groupBy: ($ {groupingToString(inputType, grouping)} ), " + } else { + "" + } + } orderBy: (eventtime), window: (unbounded), " + + s"select: ($ { + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + } ))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + } + + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream [Row] = { + + val config = tableEnv.getConfig + val groupingKeys = grouping.toArray — End diff – Over Window should partitionKeys better.
          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_r102635310

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowRule.scala —
          @@ -0,0 +1,87 @@
          +/*
          + * 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.plan.rules.datastream
          +
          +import org.apache.calcite.plan.volcano.RelSubset
          +import org.apache.calcite.plan.

          {Convention, RelOptRule, RelTraitSet}

          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.convert.ConverterRule
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamConvention, DataStreamSlideEventTimeRowAgg}

          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.plan.logical.rel.LogicalOverWindow
          +
          +import scala.collection.JavaConversions._
          +
          +/**
          + * Rule to convert a LogicalOverWindow into a DataStreamSlideEventTimeRowAgg.
          + */
          +class DataStreamWindowRule
          + extends ConverterRule(
          + classOf[LogicalOverWindow],
          + Convention.NONE,
          + DataStreamConvention.INSTANCE,
          + "DataStreamWindowRule")
          +{
          +
          + override def convert(rel: RelNode): RelNode = {
          + val agg: LogicalOverWindow = rel.asInstanceOf[LogicalOverWindow]
          + val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
          + val convInput: RelNode = RelOptRule.convert(agg.getInput, DataStreamConvention.INSTANCE)
          + val inputRowType = convInput.asInstanceOf[RelSubset].getOriginal.getRowType
          +
          + if (agg.groups.size > 1) {
          + for (i <- 0 until agg.groups.size - 1)
          + if (agg.groups.toString != agg.groups(i + 1).toString)

          { + throw new UnsupportedOperationException( + "Unsupport different window in the same projection") + }

          + }
          +
          + val win = agg.groups(0)
          + val rowtime = agg.isEventTime
          +
          + // map agg func with project name
          + var aggIdx = inputRowType.getFieldCount - 1
          + val outputNames = agg.getRowType.getFieldNames
          + val namedAgg =
          + for (i <- 0 until agg.groups.size; aggCalls = agg.groups.getAggregateCalls(agg);
          — End diff –

          IMHO. Maybe we can use a layer of iteration, because we only have one of the same window.

          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_r102635310 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowRule.scala — @@ -0,0 +1,87 @@ +/* + * 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.plan.rules.datastream + +import org.apache.calcite.plan.volcano.RelSubset +import org.apache.calcite.plan. {Convention, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.calcite.rel.core.AggregateCall +import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.plan.nodes.datastream. {DataStreamConvention, DataStreamSlideEventTimeRowAgg} +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.plan.logical.rel.LogicalOverWindow + +import scala.collection.JavaConversions._ + +/** + * Rule to convert a LogicalOverWindow into a DataStreamSlideEventTimeRowAgg. + */ +class DataStreamWindowRule + extends ConverterRule( + classOf [LogicalOverWindow] , + Convention.NONE, + DataStreamConvention.INSTANCE, + "DataStreamWindowRule") +{ + + override def convert(rel: RelNode): RelNode = { + val agg: LogicalOverWindow = rel.asInstanceOf [LogicalOverWindow] + val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE) + val convInput: RelNode = RelOptRule.convert(agg.getInput, DataStreamConvention.INSTANCE) + val inputRowType = convInput.asInstanceOf [RelSubset] .getOriginal.getRowType + + if (agg.groups.size > 1) { + for (i <- 0 until agg.groups.size - 1) + if (agg.groups .toString != agg.groups(i + 1).toString) { + throw new UnsupportedOperationException( + "Unsupport different window in the same projection") + } + } + + val win = agg.groups(0) + val rowtime = agg.isEventTime + + // map agg func with project name + var aggIdx = inputRowType.getFieldCount - 1 + val outputNames = agg.getRowType.getFieldNames + val namedAgg = + for (i <- 0 until agg.groups.size; aggCalls = agg.groups .getAggregateCalls(agg); — End diff – IMHO. Maybe we can use a layer of iteration, because we only have one of the same window.
          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_r102634122

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase

          { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2))
          + }
          +
          + /** test sliding event-time unbounded window with later record **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithLater(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          — End diff –

          Can you add some non-agg fields? That can show the different with groupWindow.

          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_r102634122 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testSlideEventTimeUnboundWindowWithGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2)) + } + + /** test sliding event-time unbounded window with later record **/ + @Test + def testSlideEventTimeUnboundWindowWithLater(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + — End diff – Can you add some non-agg fields? That can show the different with groupWindow.
          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_r102632122

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowRule.scala —
          @@ -0,0 +1,87 @@
          +/*
          + * 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.plan.rules.datastream
          +
          +import org.apache.calcite.plan.volcano.RelSubset
          +import org.apache.calcite.plan.

          {Convention, RelOptRule, RelTraitSet}

          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.convert.ConverterRule
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.plan.nodes.datastream.

          {DataStreamConvention, DataStreamSlideEventTimeRowAgg}

          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.plan.logical.rel.LogicalOverWindow
          +
          +import scala.collection.JavaConversions._
          +
          +/**
          + * Rule to convert a LogicalOverWindow into a DataStreamSlideEventTimeRowAgg.
          + */
          +class DataStreamWindowRule
          + extends ConverterRule(
          + classOf[LogicalOverWindow],
          + Convention.NONE,
          + DataStreamConvention.INSTANCE,
          + "DataStreamWindowRule")
          +{
          +
          + override def convert(rel: RelNode): RelNode = {
          + val agg: LogicalOverWindow = rel.asInstanceOf[LogicalOverWindow]
          + val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
          + val convInput: RelNode = RelOptRule.convert(agg.getInput, DataStreamConvention.INSTANCE)
          + val inputRowType = convInput.asInstanceOf[RelSubset].getOriginal.getRowType
          +
          + if (agg.groups.size > 1) {
          + for (i <- 0 until agg.groups.size - 1)
          — End diff –

          `agg.groups.size > 1 enough to determine whether there are many different windows, what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r102632122 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowRule.scala — @@ -0,0 +1,87 @@ +/* + * 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.plan.rules.datastream + +import org.apache.calcite.plan.volcano.RelSubset +import org.apache.calcite.plan. {Convention, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.calcite.rel.core.AggregateCall +import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.plan.nodes.datastream. {DataStreamConvention, DataStreamSlideEventTimeRowAgg} +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.plan.logical.rel.LogicalOverWindow + +import scala.collection.JavaConversions._ + +/** + * Rule to convert a LogicalOverWindow into a DataStreamSlideEventTimeRowAgg. + */ +class DataStreamWindowRule + extends ConverterRule( + classOf [LogicalOverWindow] , + Convention.NONE, + DataStreamConvention.INSTANCE, + "DataStreamWindowRule") +{ + + override def convert(rel: RelNode): RelNode = { + val agg: LogicalOverWindow = rel.asInstanceOf [LogicalOverWindow] + val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE) + val convInput: RelNode = RelOptRule.convert(agg.getInput, DataStreamConvention.INSTANCE) + val inputRowType = convInput.asInstanceOf [RelSubset] .getOriginal.getRowType + + if (agg.groups.size > 1) { + for (i <- 0 until agg.groups.size - 1) — End diff – `agg.groups.size > 1 enough to determine whether there are many different windows, what do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/LogicalWindowRule.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.plan.rules.datastream
          +
          +import org.apache.calcite.plan.hep.HepRelVertex
          +import org.apache.calcite.plan.

          {RelOptRule, RelOptRuleCall}

          +import org.apache.calcite.rel.logical.

          {LogicalProject, LogicalWindow}

          +import org.apache.calcite.rex.

          {RexCall, RexNode}

          +import org.apache.flink.table.functions.EventTimeExtractor
          +import org.apache.flink.table.plan.logical.rel.LogicalOverWindow
          +
          +import scala.collection.JavaConversions._
          +
          +/**
          + * Rule to check whether rowtime or proctime.
          + */
          +class LogicalWindowRule
          — End diff –

          Have you refer to FLINK-5884 and reference FLINK-5884'design to consider?(https://docs.google.com/document/d/1JRXm09x_wKst6z6UXdCGF9tgF1ueOAsFiQwahR72vbc/edit) Or we can wait for FLINK-5884 implementation, then rebase the code?

          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_r102632917 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/LogicalWindowRule.scala — @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.hep.HepRelVertex +import org.apache.calcite.plan. {RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.logical. {LogicalProject, LogicalWindow} +import org.apache.calcite.rex. {RexCall, RexNode} +import org.apache.flink.table.functions.EventTimeExtractor +import org.apache.flink.table.plan.logical.rel.LogicalOverWindow + +import scala.collection.JavaConversions._ + +/** + * Rule to check whether rowtime or proctime. + */ +class LogicalWindowRule — End diff – Have you refer to FLINK-5884 and reference FLINK-5884 'design to consider?( https://docs.google.com/document/d/1JRXm09x_wKst6z6UXdCGF9tgF1ueOAsFiQwahR72vbc/edit ) Or we can wait for FLINK-5884 implementation, then rebase the code?
          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_r102634927

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalEventTimeRowWindowAssigner.java —
          @@ -0,0 +1,113 @@
          +/**
          + * 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.streaming.api.windowing.assigners;
          +
          +import org.apache.flink.annotation.Internal;
          +import org.apache.flink.annotation.PublicEvolving;
          +import org.apache.flink.api.common.ExecutionConfig;
          +import org.apache.flink.api.common.typeutils.TypeSerializer;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.api.windowing.triggers.Trigger;
          +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
          +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
          +
          +import java.util.Collection;
          +import java.util.Collections;
          +
          +/**
          + * A

          {@link WindowAssigner}

          that assigns all elements to the same global row window.
          + *
          + */
          +@PublicEvolving
          +public class GlobalEventTimeRowWindowAssigner extends WindowAssigner<Object, GlobalWindow> {
          — End diff –

          In this way we can not handle the order of data. just like your test case `testSlideEventTimeUnboundWindowWithLater`

          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_r102634927 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalEventTimeRowWindowAssigner.java — @@ -0,0 +1,113 @@ +/** + * 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.streaming.api.windowing.assigners; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow; + +import java.util.Collection; +import java.util.Collections; + +/** + * A {@link WindowAssigner} that assigns all elements to the same global row window. + * + */ +@PublicEvolving +public class GlobalEventTimeRowWindowAssigner extends WindowAssigner<Object, GlobalWindow> { — End diff – In this way we can not handle the order of data. just like your test case `testSlideEventTimeUnboundWindowWithLater`
          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_r102631105

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala —
          @@ -52,9 +53,14 @@ class IncrementalAggregateReduceFunction(
          // and directly merge value1 and value2.
          val accumulatorRow = new Row(intermediateRowArity)

          • // copy all fields of value1 into accumulatorRow
          • (0 until intermediateRowArity)
          • .foreach(i => accumulatorRow.setField(i, value1.getField))
            + // copy non agg fields of value2 into accumulatorRow
            + (0 until aggOffset)
            + .foreach(i => accumulatorRow.setField(i, value2.getField))
            +
            + // copy agg fields of value1 into accumulatorRow
              • End diff –

          This class will be shared groupwindow and overwindow, when groupwindow do not need to set the value of the non-agg, What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r102631105 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala — @@ -52,9 +53,14 @@ class IncrementalAggregateReduceFunction( // and directly merge value1 and value2. val accumulatorRow = new Row(intermediateRowArity) // copy all fields of value1 into accumulatorRow (0 until intermediateRowArity) .foreach(i => accumulatorRow.setField(i, value1.getField )) + // copy non agg fields of value2 into accumulatorRow + (0 until aggOffset) + .foreach(i => accumulatorRow.setField(i, value2.getField )) + + // copy agg fields of value1 into accumulatorRow End diff – This class will be shared groupwindow and overwindow, when groupwindow do not need to set the value of the non-agg, What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode =

          { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + }

          +
          + override def toString: String = {
          + val inputFields = new Array[Int](inputType.getFieldCount)
          + for (i <- 0 until inputType.getFieldCount)
          + inputFields = i
          + s"Aggregate(${
          + if (!grouping.isEmpty) {
          + s"groupBy: ($

          {groupingToString(inputType, grouping)}), "
          + } else { + "" + }
          + } orderBy: (eventtime), window: (unbounded), " +
          + s"select: (${ + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + }))"
          + }
          +
          + override def explainTerms(pw: RelWriter): RelWriter = { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + }
          +
          + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
          +
          + val config = tableEnv.getConfig
          + val groupingKeys = grouping.toArray
          + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
          +
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
          + val inputFields = new Array[Int](inputType.getFieldCount)
          + for (i <- 0 until inputType.getFieldCount)
          + inputFields = i
          +
          + val aggString = aggregationToString(
          + inputType,
          + inputFields,
          + getRowType,
          + namedAggregates,
          + namedProperties)
          +
          + val prepareOpName = s"prepare select: ($aggString)"
          + val keyedAggOpName = s"groupBy: (${groupingToString(inputType, grouping)}

          ), " +
          + s"window: (unbounded), " +
          + s"select: ($aggString)"
          + val nonKeyedAggOpName = s"window: (unbounded), select: ($aggString)"
          +
          + val mapFunction = AggregateUtil.createPrepareMapFunction(
          + namedAggregates,
          + inputFields,
          + inputType)
          +
          + val mappedInput = inputDS.map(mapFunction).name(prepareOpName)
          +
          + // do Incremental Aggregation
          + val reduceFunction = AggregateUtil.createIncrementalAggregateReduceFunction(
          + namedAggregates,
          + inputType,
          + getRowType,
          + inputFields)
          — End diff –

          Do we need a parameter to distinguish between gorupWindow and OverWindow?

          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_r102631831 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + } + + override def toString: String = { + val inputFields = new Array [Int] (inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields = i + s"Aggregate(${ + if (!grouping.isEmpty) { + s"groupBy: ($ {groupingToString(inputType, grouping)}), " + } else { + "" + } + } orderBy: (eventtime), window: (unbounded), " + + s"select: (${ + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + }))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + } + + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream [Row] = { + + val config = tableEnv.getConfig + val groupingKeys = grouping.toArray + val inputDS = input.asInstanceOf [DataStreamRel] .translateToPlan(tableEnv) + + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType) + val inputFields = new Array [Int] (inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields = i + + val aggString = aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + + val prepareOpName = s"prepare select: ($aggString)" + val keyedAggOpName = s"groupBy: (${groupingToString(inputType, grouping)} ), " + + s"window: (unbounded), " + + s"select: ($aggString)" + val nonKeyedAggOpName = s"window: (unbounded), select: ($aggString)" + + val mapFunction = AggregateUtil.createPrepareMapFunction( + namedAggregates, + inputFields, + inputType) + + val mappedInput = inputDS.map(mapFunction).name(prepareOpName) + + // do Incremental Aggregation + val reduceFunction = AggregateUtil.createIncrementalAggregateReduceFunction( + namedAggregates, + inputType, + getRowType, + inputFields) — End diff – Do we need a parameter to distinguish between gorupWindow and OverWindow?
          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_r102633283

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -395,7 +395,8 @@ object AggregateUtil {
          val reduceFunction = new IncrementalAggregateReduceFunction(
          aggregates,
          groupingOffsetMapping,

          • intermediateRowArity)
            + intermediateRowArity,
            + groupings.length)
              • End diff –

          Can we distinguish between groupwWindow'grouping and overWindow'partition?

          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_r102633283 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -395,7 +395,8 @@ object AggregateUtil { val reduceFunction = new IncrementalAggregateReduceFunction( aggregates, groupingOffsetMapping, intermediateRowArity) + intermediateRowArity, + groupings.length) End diff – Can we distinguish between groupwWindow'grouping and overWindow'partition?
          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_r102631646

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode =

          { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + }

          +
          + override def toString: String = {
          + val inputFields = new Array[Int](inputType.getFieldCount)
          + for (i <- 0 until inputType.getFieldCount)
          + inputFields = i
          + s"Aggregate(${
          + if (!grouping.isEmpty) {
          + s"groupBy: ($

          {groupingToString(inputType, grouping)}

          ), "
          + } else

          { + "" + }

          + } orderBy: (eventtime), window: (unbounded), " +
          + s"select: ($

          { + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + }

          ))"
          + }
          +
          + override def explainTerms(pw: RelWriter): RelWriter =

          { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + }

          +
          + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
          +
          + val config = tableEnv.getConfig
          + val groupingKeys = grouping.toArray
          + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
          +
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
          + val inputFields = new Array[Int](inputType.getFieldCount)
          — End diff –

          Can you consider this way?
          ` val inputFields = (for(i <- 0 until inputType.getFieldCount) yield i).toArray`

          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_r102631646 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + } + + override def toString: String = { + val inputFields = new Array [Int] (inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields = i + s"Aggregate(${ + if (!grouping.isEmpty) { + s"groupBy: ($ {groupingToString(inputType, grouping)} ), " + } else { + "" + } + } orderBy: (eventtime), window: (unbounded), " + + s"select: ($ { + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + } ))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + } + + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream [Row] = { + + val config = tableEnv.getConfig + val groupingKeys = grouping.toArray + val inputDS = input.asInstanceOf [DataStreamRel] .translateToPlan(tableEnv) + + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType) + val inputFields = new Array [Int] (inputType.getFieldCount) — End diff – Can you consider this way? ` val inputFields = (for(i <- 0 until inputType.getFieldCount) yield i).toArray`
          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_r102634209

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase

          { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2))
          + }
          +
          + /** test sliding event-time unbounded window with later record **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithLater(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setParallelism(1)
          — End diff –

          Can you remove this config? Event-time should handle this situation very well, because the data carries the time. What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r102634209 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testSlideEventTimeUnboundWindowWithGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2)) + } + + /** test sliding event-time unbounded window with later record **/ + @Test + def testSlideEventTimeUnboundWindowWithLater(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setParallelism(1) — End diff – Can you remove this config? Event-time should handle this situation very well, because the data carries the time. What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
          + new DataStreamSlideEventTimeRowAgg(
          + namedProperties,
          + cluster,
          + traitSet,
          + inputs.get(0),
          + namedAggregates,
          + getRowType,
          + inputType,
          + grouping)
          — End diff –

          IMO. Over Window should partitionKeys better.

          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_r102631440 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) — End diff – IMO. Over Window should partitionKeys better.
          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_r102649307

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
          + new DataStreamSlideEventTimeRowAgg(
          + namedProperties,
          + cluster,
          + traitSet,
          + inputs.get(0),
          + namedAggregates,
          + getRowType,
          + inputType,
          + grouping)
          — End diff –

          Thanks very much for the review.
          1. The design of check whether eventtime is base on https://issues.apache.org/jira/browse/FLINK-5624 which support rowtime() as a built-in function like proctime() in FLINK-570, i think we can do in this way to support overwindow development current period, and modify once the issue done, what do you think?
          2. I check whether the current data is out of order in WindowOperator isLate function, and now just discard if islate.
          3. Yes, we should distinguish partitionkey and groupkey, i will fix 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_r102649307 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) — End diff – Thanks very much for the review. 1. The design of check whether eventtime is base on https://issues.apache.org/jira/browse/FLINK-5624 which support rowtime() as a built-in function like proctime() in FLINK-570 , i think we can do in this way to support overwindow development current period, and modify once the issue done, what do you think? 2. I check whether the current data is out of order in WindowOperator isLate function, and now just discard if islate. 3. Yes, we should distinguish partitionkey and groupkey, i will fix it.
          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_r102650085

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode =

          { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + }

          +
          + override def toString: String = {
          + val inputFields = new Array[Int](inputType.getFieldCount)
          + for (i <- 0 until inputType.getFieldCount)
          + inputFields = i
          + s"Aggregate(${
          + if (!grouping.isEmpty) {
          + s"groupBy: ($

          {groupingToString(inputType, grouping)}

          ), "
          + } else

          { + "" + }

          + } orderBy: (eventtime), window: (unbounded), " +
          + s"select: ($

          { + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + }

          ))"
          + }
          +
          + override def explainTerms(pw: RelWriter): RelWriter =

          { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + }

          +
          + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
          +
          + val config = tableEnv.getConfig
          + val groupingKeys = grouping.toArray
          + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
          +
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
          + val inputFields = new Array[Int](inputType.getFieldCount)
          — End diff –

          Yes, it's much better, i will update 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_r102650085 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) + } + + override def toString: String = { + val inputFields = new Array [Int] (inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields = i + s"Aggregate(${ + if (!grouping.isEmpty) { + s"groupBy: ($ {groupingToString(inputType, grouping)} ), " + } else { + "" + } + } orderBy: (eventtime), window: (unbounded), " + + s"select: ($ { + aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties) + } ))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + val inputFields = new Array[Int](inputType.getFieldCount) + for (i <- 0 until inputType.getFieldCount) + inputFields(i) = i + super.explainTerms(pw) + .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty) + .item("orderBy", "eventtime") + .item("window", "unbounded") + .item("select", aggregationToString( + inputType, + inputFields, + getRowType, + namedAggregates, + namedProperties)) + } + + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream [Row] = { + + val config = tableEnv.getConfig + val groupingKeys = grouping.toArray + val inputDS = input.asInstanceOf [DataStreamRel] .translateToPlan(tableEnv) + + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType) + val inputFields = new Array [Int] (inputType.getFieldCount) — End diff – Yes, it's much better, i will update it.
          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_r102652877

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala —
          @@ -52,9 +53,14 @@ class IncrementalAggregateReduceFunction(
          // and directly merge value1 and value2.
          val accumulatorRow = new Row(intermediateRowArity)

          • // copy all fields of value1 into accumulatorRow
          • (0 until intermediateRowArity)
          • .foreach(i => accumulatorRow.setField(i, value1.getField))
            + // copy non agg fields of value2 into accumulatorRow
            + (0 until aggOffset)
            + .foreach(i => accumulatorRow.setField(i, value2.getField))
            +
            + // copy agg fields of value1 into accumulatorRow
              • End diff –

          IMO, groupWindow's non-agg fields is group fields, and whether copy from value1 or value2 is the same, 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_r102652877 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala — @@ -52,9 +53,14 @@ class IncrementalAggregateReduceFunction( // and directly merge value1 and value2. val accumulatorRow = new Row(intermediateRowArity) // copy all fields of value1 into accumulatorRow (0 until intermediateRowArity) .foreach(i => accumulatorRow.setField(i, value1.getField )) + // copy non agg fields of value2 into accumulatorRow + (0 until aggOffset) + .foreach(i => accumulatorRow.setField(i, value2.getField )) + + // copy agg fields of value1 into accumulatorRow End diff – IMO, groupWindow's non-agg fields is group fields, and whether copy from value1 or value2 is the same, what do you think?
          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_r102652914

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase

          { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2))
          + }
          +
          + /** test sliding event-time unbounded window with later record **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithLater(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setParallelism(1)
          — End diff –

          Oh yes, i will remove 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_r102652914 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testSlideEventTimeUnboundWindowWithGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2)) + } + + /** test sliding event-time unbounded window with later record **/ + @Test + def testSlideEventTimeUnboundWindowWithLater(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setParallelism(1) — End diff – Oh yes, i will remove it.
          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_r102652926

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase

          { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2))
          + }
          +
          + /** test sliding event-time unbounded window with later record **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithLater(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          — End diff –

          The testSlideEventTimeUnboundWindowWithGroup has include non-agg field.

          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_r102652926 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testSlideEventTimeUnboundWindowWithGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2)) + } + + /** test sliding event-time unbounded window with later record **/ + @Test + def testSlideEventTimeUnboundWindowWithLater(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + — End diff – The testSlideEventTimeUnboundWindowWithGroup has include non-agg field.
          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_r102652936

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalEventTimeRowWindowAssigner.java —
          @@ -0,0 +1,113 @@
          +/**
          + * 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.streaming.api.windowing.assigners;
          +
          +import org.apache.flink.annotation.Internal;
          +import org.apache.flink.annotation.PublicEvolving;
          +import org.apache.flink.api.common.ExecutionConfig;
          +import org.apache.flink.api.common.typeutils.TypeSerializer;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.api.windowing.triggers.Trigger;
          +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
          +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
          +
          +import java.util.Collection;
          +import java.util.Collections;
          +
          +/**
          + * A

          {@link WindowAssigner}

          that assigns all elements to the same global row window.
          + *
          + */
          +@PublicEvolving
          +public class GlobalEventTimeRowWindowAssigner extends WindowAssigner<Object, GlobalWindow> {
          — End diff –

          This windowassigner is aim to keep the currentelement eventtime and current max timestamp, if new element timestamp is before the current max timestamp, i will just discard 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_r102652936 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalEventTimeRowWindowAssigner.java — @@ -0,0 +1,113 @@ +/** + * 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.streaming.api.windowing.assigners; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow; + +import java.util.Collection; +import java.util.Collections; + +/** + * A {@link WindowAssigner} that assigns all elements to the same global row window. + * + */ +@PublicEvolving +public class GlobalEventTimeRowWindowAssigner extends WindowAssigner<Object, GlobalWindow> { — End diff – This windowassigner is aim to keep the currentelement eventtime and current max timestamp, if new element timestamp is before the current max timestamp, i will just discard 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_r102922045

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala —
          @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase

          { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) }

          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2))
          + }
          +
          + /** test sliding event-time unbounded window with later record **/
          + @Test
          + def testSlideEventTimeUnboundWindowWithLater(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.setParallelism(1)
          — End diff –

          When you remove this configuration, run many times will get different results, such as:
          I have added a time column for the convenience of observation `SUM(a) -> d, SUM(a) `:

          The first time:
          ```
          1400005,2
          1400010,6
          ```
          The second time
          ```
          1400000,1
          1400010,5
          ```
          The third time
          ```
          1300000,3
          1400005,5
          1400010,9
          ```
          Do you have that happen?If so, please check and fix the bug?

          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_r102922045 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -171,4 +175,98 @@ class SqlITCase extends StreamingMultipleProgramsTestBase { val expected = mutable.MutableList("Hello", "Hello world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testSlideEventTimeUnboundWindowWithGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(System.currentTimeMillis) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2)) + } + + /** test sliding event-time unbounded window with later record **/ + @Test + def testSlideEventTimeUnboundWindowWithLater(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.setParallelism(1) — End diff – When you remove this configuration, run many times will get different results, such as: I have added a time column for the convenience of observation `SUM(a) -> d, SUM(a) `: The first time: ``` 1400005,2 1400010,6 ``` The second time ``` 1400000,1 1400010,5 ``` The third time ``` 1300000,3 1400005,5 1400010,9 ``` Do you have that happen?If so, please check and fix the bug?
          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_r102925177

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java —
          @@ -563,7 +590,14 @@ private void emitWindowContents(W window, ACC contents) throws Exception {

          • of the given window.
            */
            protected boolean isLate(W window) {
          • return (windowAssigner.isEventTime() && (cleanupTime(window) <= internalTimerService.currentWatermark()));
            + if (windowAssigner.isEventTime()) {
            + if (windowAssigner instanceof GlobalEventTimeRowWindowAssigner) {
            + return windowAssignerContext.getCurrentElementTime() < windowAssignerContext.getCurrentMaxTime();
              • End diff –

          I think this logic does not work very well if the data arrives is out of order. How do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r102925177 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java — @@ -563,7 +590,14 @@ private void emitWindowContents(W window, ACC contents) throws Exception { of the given window. */ protected boolean isLate(W window) { return (windowAssigner.isEventTime() && (cleanupTime(window) <= internalTimerService.currentWatermark())); + if (windowAssigner.isEventTime()) { + if (windowAssigner instanceof GlobalEventTimeRowWindowAssigner) { + return windowAssignerContext.getCurrentElementTime() < windowAssignerContext.getCurrentMaxTime(); End diff – I think this logic does not work very well if the data arrives is out of order. How do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
          + new DataStreamSlideEventTimeRowAgg(
          + namedProperties,
          + cluster,
          + traitSet,
          + inputs.get(0),
          + namedAggregates,
          + getRowType,
          + inputType,
          + grouping)
          — End diff –

          I think `I check whether the current data is out of order in WindowOperator isLate function, and now just discard if islate.` will not work well. Because the logic in this method is:
          ```
          if (windowAssigner instanceof GlobalEventTimeRowWindowAssigner)

          { return windowAssignerContext.getCurrentElementTime() < windowAssignerContext.getCurrentMaxTime(); }


          ```
          e.g. Test Data:
          ```
          1, 1L, "Hi", 1400000L
          2, 2L, "Hello", 1400005L
          3, 2L, "Hello w", 1300000
          4, 3L, "Hello world", 1400010L
          ```
          You do not know which element first comes, so you will get different results every time you run it,Just like:
          `SELECT` d, SUM(a) over (order by rowtime() range between unbounded preceding and current row) from T1`
          You can get the following results:
          The first time:
          ```
          1400005,2
          1400010,6
          ```
          The second time
          ```
          1400000,1
          1400010,5
          ```
          The third time
          ```
          1300000,3
          1400005,5
          1400010,9
          ```
          So,IMHO. Event-time over must handle the situation above. How do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r102927451 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) — End diff – I think `I check whether the current data is out of order in WindowOperator isLate function, and now just discard if islate.` will not work well. Because the logic in this method is: ``` if (windowAssigner instanceof GlobalEventTimeRowWindowAssigner) { return windowAssignerContext.getCurrentElementTime() < windowAssignerContext.getCurrentMaxTime(); } ``` e.g. Test Data: ``` 1, 1L, "Hi", 1400000L 2, 2L, "Hello", 1400005L 3, 2L, "Hello w", 1300000 4, 3L, "Hello world", 1400010L ``` You do not know which element first comes, so you will get different results every time you run it,Just like: `SELECT` d, SUM(a) over (order by rowtime() range between unbounded preceding and current row) from T1` You can get the following results: The first time: ``` 1400005,2 1400010,6 ``` The second time ``` 1400000,1 1400010,5 ``` The third time ``` 1300000,3 1400005,5 1400010,9 ``` So,IMHO. Event-time over must handle the situation above. How do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala —
          @@ -52,9 +53,14 @@ class IncrementalAggregateReduceFunction(
          // and directly merge value1 and value2.
          val accumulatorRow = new Row(intermediateRowArity)

          • // copy all fields of value1 into accumulatorRow
          • (0 until intermediateRowArity)
          • .foreach(i => accumulatorRow.setField(i, value1.getField))
            + // copy non agg fields of value2 into accumulatorRow
            + (0 until aggOffset)
            + .foreach(i => accumulatorRow.setField(i, value2.getField))
            +
            + // copy agg fields of value1 into accumulatorRow
              • End diff –

          I mean in the `groupWindow` we only need to one times traverse assignment.

          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_r102928695 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala — @@ -52,9 +53,14 @@ class IncrementalAggregateReduceFunction( // and directly merge value1 and value2. val accumulatorRow = new Row(intermediateRowArity) // copy all fields of value1 into accumulatorRow (0 until intermediateRowArity) .foreach(i => accumulatorRow.setField(i, value1.getField )) + // copy non agg fields of value2 into accumulatorRow + (0 until aggOffset) + .foreach(i => accumulatorRow.setField(i, value2.getField )) + + // copy agg fields of value1 into accumulatorRow End diff – I mean in the `groupWindow` we only need to one times traverse assignment.
          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_r103125615

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
          + new DataStreamSlideEventTimeRowAgg(
          + namedProperties,
          + cluster,
          + traitSet,
          + inputs.get(0),
          + namedAggregates,
          + getRowType,
          + inputType,
          + grouping)
          — End diff –

          Yes, we don' know which element will come first. Thus i think we should use the retraction solution like we discuss before.
          ```
          1400000 -> (1400000, 1)
          1400005-> (1400005, 3)
          1300000-> (1300000, 3), (1400000, 4), (1400005, 6)
          1400010-> (1400010, 10)
          ```
          And we just persist the aggregation status of every element for one watermark period, when watermark come, the following record which timestamp before the watermark will be discarded.
          And the problem is one element may generate multiple output in different time, like the example, the first element will output one when it arrive, and will ouput another again when the third element come, it need the following operator or user to do an update operation.
          What's your opinion?

          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_r103125615 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) — End diff – Yes, we don' know which element will come first. Thus i think we should use the retraction solution like we discuss before. ``` 1400000 -> (1400000, 1) 1400005-> (1400005, 3) 1300000-> (1300000, 3), (1400000, 4), (1400005, 6) 1400010-> (1400010, 10) ``` And we just persist the aggregation status of every element for one watermark period, when watermark come, the following record which timestamp before the watermark will be discarded. And the problem is one element may generate multiple output in different time, like the example, the first element will output one when it arrive, and will ouput another again when the third element come, it need the following operator or user to do an update operation. What's your opinion?
          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_r103404131

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala —
          @@ -0,0 +1,179 @@
          +/*
          + * 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.plan.nodes.datastream
          +
          +import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.rel.`type`.RelDataType
          +import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rel.

          {RelNode, RelWriter, SingleRel}

          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.table.calcite.

          {FlinkRelBuilder, FlinkTypeFactory}

          +import FlinkRelBuilder.NamedWindowProperty
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil._
          +import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.streaming.api.datastream.

          {AllWindowedStream, DataStream, WindowedStream}

          +import org.apache.flink.streaming.api.windowing.assigners._
          +import org.apache.flink.streaming.api.windowing.windows.

          {Window => DataStreamWindow}

          +import org.apache.flink.table.api.StreamTableEnvironment
          +import org.apache.flink.table.plan.nodes.CommonAggregate
          +
          +class DataStreamSlideEventTimeRowAgg(
          + namedProperties: Seq[NamedWindowProperty],
          + cluster: RelOptCluster,
          + traitSet: RelTraitSet,
          + inputNode: RelNode,
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + rowRelDataType: RelDataType,
          + inputType: RelDataType,
          + grouping: Array[Int])
          + extends SingleRel(cluster, traitSet, inputNode)
          + with CommonAggregate
          + with DataStreamRel {
          +
          + override def deriveRowType(): RelDataType = rowRelDataType
          +
          + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
          + new DataStreamSlideEventTimeRowAgg(
          + namedProperties,
          + cluster,
          + traitSet,
          + inputs.get(0),
          + namedAggregates,
          + getRowType,
          + inputType,
          + grouping)
          — End diff –

          In fact, we can discard late events, but we must have a strategy to define what kind of element is late.
          Of course, the current implementation is also a strategy to assess the delay event, but this strategy will lose too many events, and data calculation results are unpredictable, not playback. This is unacceptable in the production situation. What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r103404131 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSlideEventTimeRowAgg.scala — @@ -0,0 +1,179 @@ +/* + * 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.plan.nodes.datastream + +import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rel. {RelNode, RelWriter, SingleRel} +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.table.calcite. {FlinkRelBuilder, FlinkTypeFactory} +import FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.runtime.aggregate.AggregateUtil._ +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.streaming.api.datastream. {AllWindowedStream, DataStream, WindowedStream} +import org.apache.flink.streaming.api.windowing.assigners._ +import org.apache.flink.streaming.api.windowing.windows. {Window => DataStreamWindow} +import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.plan.nodes.CommonAggregate + +class DataStreamSlideEventTimeRowAgg( + namedProperties: Seq [NamedWindowProperty] , + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + rowRelDataType: RelDataType, + inputType: RelDataType, + grouping: Array [Int] ) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel { + + override def deriveRowType(): RelDataType = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List [RelNode] ): RelNode = { + new DataStreamSlideEventTimeRowAgg( + namedProperties, + cluster, + traitSet, + inputs.get(0), + namedAggregates, + getRowType, + inputType, + grouping) — End diff – In fact, we can discard late events, but we must have a strategy to define what kind of element is late. Of course, the current implementation is also a strategy to assess the delay event, but this strategy will lose too many events, and data calculation results are unpredictable, not playback. This is unacceptable in the production situation. What do you think?
          Hide
          Yuhong_kyo hongyuhong added a comment -

          Hi Fabian Hueske, as we talked before, We simply discard late arriving records.
          I plan to use watermark, and discard record before current watermark.
          like

          A(Time:10) -> agg(A)
          B(Time:11) -> agg(A,B)
          C(Time:12) -> agg(A,B,C)
          -------watermark(Time:12)----------
          D(Time:11) -> _ //just discard
          E(Time:13) -> agg(A,B,C,E)
          

          But as the records that not be discarded, the result will be different according to the order they come,
          such as

          A(Time:10) -> agg(A)               B(Time:11) -> agg(B)             C(Time:12) -> agg(C) 
          B(Time:11) -> agg(A,B)             A(Time:10) -> agg(A)             B(Time:11) -> agg(B)
          C(Time:12) -> agg(A,B,C)           C(Time:12) -> agg(B,C)           A(Time:10) -> agg(A) 
          

          so i think a retraction operation may be needed to make sure they have same results.

          B(Time:11) -> agg(B) 
          A(Time:10) -> agg(A), agg(A,B)
          C(Time:12) -> agg(A,B,C)
          

          What do you thnk? hope can give some suggestion.

          Show
          Yuhong_kyo hongyuhong added a comment - Hi Fabian Hueske , as we talked before, We simply discard late arriving records. I plan to use watermark, and discard record before current watermark. like A(Time:10) -> agg(A) B(Time:11) -> agg(A,B) C(Time:12) -> agg(A,B,C) -------watermark(Time:12)---------- D(Time:11) -> _ //just discard E(Time:13) -> agg(A,B,C,E) But as the records that not be discarded, the result will be different according to the order they come, such as A(Time:10) -> agg(A) B(Time:11) -> agg(B) C(Time:12) -> agg(C) B(Time:11) -> agg(A,B) A(Time:10) -> agg(A) B(Time:11) -> agg(B) C(Time:12) -> agg(A,B,C) C(Time:12) -> agg(B,C) A(Time:10) -> agg(A) so i think a retraction operation may be needed to make sure they have same results. B(Time:11) -> agg(B) A(Time:10) -> agg(A), agg(A,B) C(Time:12) -> agg(A,B,C) What do you thnk? hope can give some suggestion.
          Hide
          sunjincheng121 sunjincheng added a comment - - edited

          HI hongyuhong Let me share some ideas about this.

          First of all,
          I suggest use processFunction to implement this JIRA. Because processFunction can manager `TimerService` which can control trigger window calculation. It's very powerful.( Actually, mentioned in the Fabian Hueske's ML )

          secondly,
          I suggest manage StateBackend in processFunction, then STATE together with the TIMER, may be able to solve this problem.
          Maybe we need a more detailed design, here is just a suggestion, for reference only. I believe that if there is a mistake here, Fabian Hueske will do the correction.
          What do you think?
          Best,
          SunJincheng

          Show
          sunjincheng121 sunjincheng added a comment - - edited HI hongyuhong Let me share some ideas about this. First of all, I suggest use processFunction to implement this JIRA. Because processFunction can manager `TimerService` which can control trigger window calculation. It's very powerful.( Actually, mentioned in the Fabian Hueske 's ML ) secondly, I suggest manage StateBackend in processFunction, then STATE together with the TIMER, may be able to solve this problem. Maybe we need a more detailed design, here is just a suggestion, for reference only. I believe that if there is a mistake here, Fabian Hueske will do the correction. What do you think? Best, SunJincheng
          Hide
          Yuhong_kyo hongyuhong added a comment -

          Hi sunjincheng, i agree with you, i have already rewrite this use processFunction, in order to maintain consistency with unbounded proctime, maybe i should wait for your merge, or i can push my implemention first for review?

          Show
          Yuhong_kyo hongyuhong added a comment - Hi sunjincheng , i agree with you, i have already rewrite this use processFunction, in order to maintain consistency with unbounded proctime, maybe i should wait for your merge, or i can push my implemention first for review?
          Hide
          sunjincheng121 sunjincheng added a comment -

          Hi, hongyuhong, sounds good to me. The [partitioned] PR#3397 already merged. [non-partitioned] PR#3491 today should be able to merge. Then you can rebase your code on master.

          Thanks,
          SunJincheng

          Show
          sunjincheng121 sunjincheng added a comment - Hi, hongyuhong , sounds good to me. The [partitioned] PR #3397 already merged. [non-partitioned] PR #3491 today should be able to merge. Then you can rebase your code on master. Thanks, SunJincheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hongyuhong commented on the issue:

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

          I have rebase and update the pr, appreciate if you can have a look.

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on the issue: https://github.com/apache/flink/pull/3386 I have rebase and update the pr, appreciate if you can have a look.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi, @hongyuhong Thanks for updated the PR. I'll fast pass over the changes, and leave feedback for you.
          Best,
          SunJIncheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3386 Hi, @hongyuhong Thanks for updated the PR. I'll fast pass over the changes, and leave feedback for you. Best, SunJIncheng
          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_r105323567

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventtimeOverProcessFunction.scala —
          @@ -0,0 +1,285 @@
          +/*
          + * 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(
          — End diff –

          I suggest that change `Eventtime` to `EventTime`, What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r105323567 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventtimeOverProcessFunction.scala — @@ -0,0 +1,285 @@ +/* + * 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( — End diff – I suggest that change `Eventtime` to `EventTime`, What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,133 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.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 {
          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)] {
          +
          + def getCurrentWatermark: Watermark = new Watermark(1300000L)
          — End diff –

          Why use a fixed value to produce watermark, can generate watermark based on data?

          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_r105322866 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.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 { + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) — End diff – Why use a fixed value to produce watermark, can generate watermark based on data?
          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_r105322751

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,133 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.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 {
          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          — End diff –

          I suggest that partitionBy `c` filed. Just a suggestion.

          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_r105322751 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.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 { + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + — End diff – I suggest that partitionBy `c` filed. Just a suggestion.
          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_r105323410

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -80,6 +81,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(
          — End diff –

          I suggest that change `Eventtime` to `EventTime`, What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r105323410 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -80,6 +81,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( — End diff – I suggest that change `Eventtime` to `EventTime`, What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,133 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.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 {
          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testWithoutPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(StreamITCase.testResults.size - 1))
          + }
          +
          + /** test sliding event-time unbounded window with later record **/
          + @Test
          + def testWithLater(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + env.getConfig.setAutoWatermarkInterval(10000);
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT d, SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall4TupleEventTimeDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[Small4Tuple] {
          + var cur = 1200000L;
          +
          + def getCurrentWatermark: Watermark = new Watermark(

          {cur += 10; cur;}

          )
          +
          + def extractTimestamp(element: Small4Tuple, previousElementTimestamp: Long): Long =
          + element.d
          — End diff –

          Can we set the value of `cur` based on the data?

          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_r105323083 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.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 { + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testWithoutPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(StreamITCase.testResults.size - 1)) + } + + /** test sliding event-time unbounded window with later record **/ + @Test + def testWithLater(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + env.getConfig.setAutoWatermarkInterval(10000); + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT d, SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall4TupleEventTimeDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [Small4Tuple] { + var cur = 1200000L; + + def getCurrentWatermark: Watermark = new Watermark( {cur += 10; cur;} ) + + def extractTimestamp(element: Small4Tuple, previousElementTimestamp: Long): Long = + element.d — End diff – Can we set the value of `cur` based on the data?
          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_r105325003

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

          It's not a good idea to use a memory data structure here because i worry about OOM problems in big data situations, I suggest use stateBackend. Unfortunately we are not currently sorting stateBackend, maybe we can think about other ways. I'm not sure, but I need to think about it and then give you feedback. what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r105325003 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventtimeOverProcessFunction.scala — @@ -0,0 +1,285 @@ +/* + * 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 – It's not a good idea to use a memory data structure here because i worry about OOM problems in big data situations, I suggest use stateBackend. Unfortunately we are not currently sorting stateBackend, maybe we can think about other ways. I'm not sure, but I need to think about it and then give you feedback. what do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,133 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.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 {
          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          — End diff –

          This test data is a bit simple, I recommend enriching some test data, such as:
          ```
          data.+=((1, 1L, "Hi"))
          data.+=((2, 2L, "Hello"))
          data.+=((3, 5L, "Hello"))
          data.+=((1, 3L, "Hello"))
          data.+=((3, 7L, "Hello world"))
          data.+=((4, 9L, "Hello world"))
          data.+=((5, 8L, "Hello world"))
          ```
          what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r105322654 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.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 { + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) — End diff – This test data is a bit simple, I recommend enriching some test data, such as: ``` data.+=((1, 1L, "Hi")) data.+=((2, 2L, "Hello")) data.+=((3, 5L, "Hello")) data.+=((1, 3L, "Hello")) data.+=((3, 7L, "Hello world")) data.+=((4, 9L, "Hello world")) data.+=((5, 8L, "Hello world")) ``` what do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventtimeOverProcessFunction.scala —
          @@ -0,0 +1,285 @@
          +/*
          + * 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 = {
          +
          + // discard later record
          + if (ctx.timestamp() < ctx.timerService().currentWatermark()) {
          + return
          — End diff –

          Do not suggest use `return` in scalar. e.g.:
          We can change
          ```
          if (a > b)

          { return }

          else

          { // do something }

          ```
          to
          ```
          if (a<=b)

          { // do something }

          ```
          What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3386#discussion_r105324505 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventtimeOverProcessFunction.scala — @@ -0,0 +1,285 @@ +/* + * 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 = { + + // discard later record + if (ctx.timestamp() < ctx.timerService().currentWatermark()) { + return — End diff – Do not suggest use `return` in scalar. e.g.: We can change ``` if (a > b) { return } else { // do something } ``` to ``` if (a<=b) { // do something } ``` What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala —
          @@ -0,0 +1,133 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.flink.table.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 {
          +
          + /** test sliding event-time unbounded window with partition by **/
          + @Test
          + def testWithPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " +
          + "unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).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 expected1 = mutable.MutableList(
          + "1,1,1", "2,2,2", "3,2,5")
          + val expected2 = mutable.MutableList(
          + "1,1,1", "2,2,5", "3,2,3")
          + assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
          + expected2.equals(StreamITCase.testResults.sorted))
          + }
          +
          + /** test sliding event-time unbounded window without partitiion by **/
          + @Test
          + def testWithoutPartition(): Unit = {
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT SUM(a) " +
          + "over (order by rowtime() range between unbounded preceding and current row) from T1"
          +
          + val t1 = StreamTestData.getSmall3TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks[(Int, Long, String)]

          { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + }

          ).toTable(tEnv).as('a, 'b, 'c)
          + tEnv.registerTable("T1", t1)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(StreamITCase.testResults.size - 1))
          — End diff –

          Can you test the results of each output?

          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_r105322914 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UnboundedRowtimeOverTest.scala — @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.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 { + + /** test sliding event-time unbounded window with partition by **/ + @Test + def testWithPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by rowtime() range between " + + "unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).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 expected1 = mutable.MutableList( + "1,1,1", "2,2,2", "3,2,5") + val expected2 = mutable.MutableList( + "1,1,1", "2,2,5", "3,2,3") + assertTrue(expected1.equals(StreamITCase.testResults.sorted) || + expected2.equals(StreamITCase.testResults.sorted)) + } + + /** test sliding event-time unbounded window without partitiion by **/ + @Test + def testWithoutPartition(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT SUM(a) " + + "over (order by rowtime() range between unbounded preceding and current row) from T1" + + val t1 = StreamTestData.getSmall3TupleDataStream(env) + .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks [(Int, Long, String)] { + + def getCurrentWatermark: Watermark = new Watermark(1300000L) + + def extractTimestamp(element: (Int, Long, String), previousElementTimestamp: Long): Long = + 1400000 + } ).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("T1", t1) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(Some("6"), StreamITCase.testResults.sorted.get(StreamITCase.testResults.size - 1)) — End diff – Can you test the results of each output?
          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_r105337481

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

          Yes, there may be some memory problem if the watermark interval is too long, but if use statebackend, the cost of serialize whole list everytime when update is too large, like you say, if there can be a sorting statebackend that can provide quikly search and update, that's great for me.

          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_r105337481 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventtimeOverProcessFunction.scala — @@ -0,0 +1,285 @@ +/* + * 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 – Yes, there may be some memory problem if the watermark interval is too long, but if use statebackend, the cost of serialize whole list everytime when update is too large, like you say, if there can be a sorting statebackend that can provide quikly search and update, that's great for me.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user hongyuhong commented on the issue:

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

          Hi @sunjincheng121, thanks very much for your suggestion, i have update the pr.

          Best,
          Hongyuhong

          Show
          githubbot ASF GitHub Bot added a comment - Github user hongyuhong commented on the issue: https://github.com/apache/flink/pull/3386 Hi @sunjincheng121, thanks very much for your suggestion, i have update the pr. Best, Hongyuhong
          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_r106188790

          — 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)
          — End diff –

          we should register event time timers and wait for `onTimer()` calls to react to watermarks.

          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_r106188790 — 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) — End diff – we should register event time timers and wait for `onTimer()` calls to react to watermarks.
          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_r106190499

          — 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)
          — End diff –

          The accumulator does not need the forwarded fields.

          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_r106190499 — 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) — End diff – The accumulator does not need the forwarded fields.
          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_r106210445

          — Diff: flink-libraries/flink-table/pom.xml —
          @@ -124,6 +124,21 @@ under the License.
          <version>$

          {project.version}

          </version>
          <scope>test</scope>
          </dependency>
          + <dependency>
          — End diff –

          add newline

          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_r106210445 — Diff: flink-libraries/flink-table/pom.xml — @@ -124,6 +124,21 @@ under the License. <version>$ {project.version} </version> <scope>test</scope> </dependency> + <dependency> — End diff – add newline
          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_r106195270

          — 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 –

          To be honest, I think this design is a bit too complicated for the current state of the Table API / SQL.
          So far, we cannot handle retractions and a result cannot be updated once emitted.
          I propose the following design:

          ```
          open()

          { // initalize a ListState[Tuple2[Long, Row]] to collect rows between watermarks // initialize a ValueState[Row] for the accumulators of the aggregation functions. }

          processElement(row: Row, ctx: Context, out: Collector) =

          { // check if row timestamp is before current watermark // if yes, return // else put the row in a ListState of Tuple2[Long, Row] (long is timestamp) and register a timer for rowTimestamp + 1 }

          // onTimer is called for each registered event-time timer when the watermark advances beyond the timer's timestamp.
          onTimer()

          { // get ListState and check size. // is size == 0, return // if size > 0 insert list into a priority queue sorted by timestamp // fetch the accumulator for the key (we hold one Row with accumulators as ValueState) // go over the priority queue, update the accumulator and emit the result. // clear list state and update accumulator }

          ```

          With this design, processing each row is cheap (inserting into ListState) and we have to do work when a new watermark arrives. The multiple calls of `onTimer()`for each watermark should be OK. We ensure that data is correctly sorted but drop late data (just like the group windows at the moment).
          Further benefits are that we let Flink handle the key and state management. So no need for key extractors or implementing the `CheckpointedFunction` interface.

          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_r106195270 — 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 – To be honest, I think this design is a bit too complicated for the current state of the Table API / SQL. So far, we cannot handle retractions and a result cannot be updated once emitted. I propose the following design: ``` open() { // initalize a ListState[Tuple2[Long, Row]] to collect rows between watermarks // initialize a ValueState[Row] for the accumulators of the aggregation functions. } processElement(row: Row, ctx: Context, out: Collector) = { // check if row timestamp is before current watermark // if yes, return // else put the row in a ListState of Tuple2[Long, Row] (long is timestamp) and register a timer for rowTimestamp + 1 } // onTimer is called for each registered event-time timer when the watermark advances beyond the timer's timestamp. onTimer() { // get ListState and check size. // is size == 0, return // if size > 0 insert list into a priority queue sorted by timestamp // fetch the accumulator for the key (we hold one Row with accumulators as ValueState) // go over the priority queue, update the accumulator and emit the result. // clear list state and update accumulator } ``` With this design, processing each row is cheap (inserting into ListState) and we have to do work when a new watermark arrives. The multiple calls of `onTimer()`for each watermark should be OK. We ensure that data is correctly sorted but drop late data (just like the group windows at the moment). Further benefits are that we let Flink handle the key and state management. So no need for key extractors or implementing the `CheckpointedFunction` interface.
          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.
          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_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_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_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_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_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_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_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_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_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 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 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 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 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 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 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 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_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 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_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 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 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 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 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 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_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_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_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_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_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_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 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 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_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_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_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_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_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_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_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_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_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_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_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_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.