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

Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL

    Details

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

      Description

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

      Queries similar to the following should be supported:

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

      The following restrictions should initially apply:

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

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

      This issue includes:

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

        Issue Links

          Activity

          Hide
          stefano.bortoli Stefano Bortoli added a comment -

          Hi Fabian, I will start working on this issue in the next days. Please assign it to me.

          Show
          stefano.bortoli Stefano Bortoli added a comment - Hi Fabian, I will start working on this issue in the next days. Please assign it to me.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user huawei-flink opened a pull request:

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

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

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [X] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [X] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [X] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

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

          $ git pull https://github.com/huawei-flink/flink FLINK-5653

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

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


          commit 72ec35a7380a4d73bd092ce14962ab2248139bae
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-01T16:15:58Z

          First implementation of ProcTime()

          commit e98c28616af1cf67d3ad3277d9cc2ca335604eca
          Author: rtudoran <rtudoran@bigdata-hp3>
          Date: 2017-02-02T10:30:40Z

          Disambiguate for the OVER BY clause, which should not be treated as a
          RexOver expression in Logical Project

          commit b7e6a673b88b6181c06071cd6c7bda55c25a62b4
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-02T12:07:11Z

          Added return to disambiguation method for rexover

          commit cda17565d5969f29b16923b631178a2cbf64791b
          Author: rtudoran <rtudoran@bigdata-hp3>
          Date: 2017-02-02T16:00:20Z

          Enable the LogicalWindow operators in query translation

          commit 4b3e54281018b83c818f91e09a5321c34bbf297b
          Author: rtudoran <rtudoran@bigdata-hp3>
          Date: 2017-02-03T14:59:39Z

          Added a DataStreamRel version that can be extended in java

          commit cc960d699db369cc8dc4e155cc5c5f6c3baf74a4
          Author: rtudoran <rtudoran@bigdata-hp3>
          Date: 2017-02-03T15:35:18Z

          Add skeleton for the implementation of the aggregates over sliding
          window with processing time and time boundaries

          commit 2390a9d3dc15afba01185c47f61a9ea830ea5acc
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-06T10:33:57Z

          committing changes with stub modifications before chekout proctime
          branch

          commit eaf4e92784dab01b17004390968ca4b1fe7c4bea
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-06T13:17:43Z

          ignore aggregation test and implemented simple proctime test

          commit 16ccd7f5bf019803ea8b53f09a126ec53a5a6d59
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-06T14:17:03Z

          Merge branch 'FLINK-5710' of https://github.com/huawei-flink/flink into FLINK-5653

          commit 10f7bc5e2086e41ec76cbabdcd069c71a491671a
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-07T09:42:41Z

          committing first key selector and utils

          commit 31060e46f78729880c03e8cab0f92ff06faec4f0
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-07T11:16:43Z

          Changed ProcTime from time to timestamp

          commit 69289bad836a5fdace271b28a15ca0e309e50b17
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-07T13:13:23Z

          Merge branch 'FLINK-5710' of https://github.com/huawei-flink/flink into FLINK-5654

          commit 3392817045ed166df5f55d22fde34cbd98c775db
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-07T13:14:50Z

          Merge branch 'FLINK-5653' of https://github.com/huawei-flink/flink into FLINK-5654

          commit d2ea0076b5e3561585c4eaea84025e50beaacf9a
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-07T09:42:41Z

          fixing linelength and other issues

          commit f29f564bb7fe7496b9f3d2f45a6b4469af559378
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-07T13:46:30Z

          Merge branch 'FLINK-5653' of https://github.com/huawei-flink/flink.git
          into FLINK-5653

          Conflicts:
          flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/DataStreamWindowRowAggregate.java
          flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/StreamGroupKeySelector.java

          commit ea145ecefc2be1bea71e995dbf39585e7fa44012
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-07T13:46:53Z

          Merge branch 'FLINK-5653' of https://github.com/huawei-flink/flink into FLINK-5654

          commit 30749fc34d96a00427d079f803d44ba7007e619b
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-08T17:52:18Z

          Added the integration of the WindowProcTime object in the convertion
          rules and enable the convertion

          commit b001ce5505833773a9143ff1f8581eb308357b6d
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-09T14:06:48Z

          pushing Util function to synchronize with other branch

          commit 41c9ed49359b2ae9ad552eb07e51c8a60c219e97
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-09T14:07:35Z

          Merge branch 'FLINK-5653' of https://github.com/huawei-flink/flink into FLINK-5654

          commit 9c0dc0b17824ec967635c93974004533db3118b9
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-09T16:57:49Z

          partition verification

          commit cb3d7050fff0e119c1e49b7b2ad5294774041091
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-10T14:48:03Z

          First version of ProcTime aggregation with Max, Min and Sum. To be
          tested in a deeper way, as well as introducing other aggregations

          commit 92d96bfe7e4dfe6426e10e59237fbae3cf772b61
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-10T15:00:22Z

          Merge branch 'FLINK-5654' of https://github.com/huawei-flink/flink into
          FLINK-5653

          1. Conflicts:
          2. flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/WindowAggregateUtil.java

          commit 2d6ea54cdfe7096fcd6bbb8ae8bbdf2cd2ba2d0c
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-10T16:53:03Z

          fixing unused imports

          commit 40f5ff80bb3023ce1125094dca135bc230ab67ba
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-28T15:52:35Z

          Adding aggregation functions in new package

          commit 65f47e99f3770a4980a3ddc412a6904ca5c69c55
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-28T15:59:08Z

          pushing first version of windowed aggregation working on keyed stream

          commit abe533b82b9e96ff63d52d405530d591ff819eb7
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-28T17:23:00Z

          first implementation of proctime row based aggregation

          commit 14ff2e9cb7e3b313d64b194956e888d8a5c3cf95
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-03-01T09:04:05Z

          renaming test class

          commit d6830b3c6be7b304a7bf7fbff464c197b8dd6591
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-03-01T13:46:09Z

          fixing style and removing aggregate time references

          commit 3cf37f0222a8c39b74b46873cc5f5ac12adabebf
          Author: rtudoran <rtudoran@bigdata-hp3>
          Date: 2017-02-02T10:30:40Z

          Disambiguate for the OVER BY clause, which should not be treated as a
          RexOver expression in Logical Project

          commit ea0f7af0cc7bc27975b6d1a2a946c9d0bf73cb73
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-02-02T12:07:11Z

          Added return to disambiguation method for rexover


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user huawei-flink opened a pull request: https://github.com/apache/flink/pull/3443 FLINK-5653 Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [X] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [X] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [X] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed You can merge this pull request into a Git repository by running: $ git pull https://github.com/huawei-flink/flink FLINK-5653 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3443.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 #3443 commit 72ec35a7380a4d73bd092ce14962ab2248139bae Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-01T16:15:58Z First implementation of ProcTime() commit e98c28616af1cf67d3ad3277d9cc2ca335604eca Author: rtudoran <rtudoran@bigdata-hp3> Date: 2017-02-02T10:30:40Z Disambiguate for the OVER BY clause, which should not be treated as a RexOver expression in Logical Project commit b7e6a673b88b6181c06071cd6c7bda55c25a62b4 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-02T12:07:11Z Added return to disambiguation method for rexover commit cda17565d5969f29b16923b631178a2cbf64791b Author: rtudoran <rtudoran@bigdata-hp3> Date: 2017-02-02T16:00:20Z Enable the LogicalWindow operators in query translation commit 4b3e54281018b83c818f91e09a5321c34bbf297b Author: rtudoran <rtudoran@bigdata-hp3> Date: 2017-02-03T14:59:39Z Added a DataStreamRel version that can be extended in java commit cc960d699db369cc8dc4e155cc5c5f6c3baf74a4 Author: rtudoran <rtudoran@bigdata-hp3> Date: 2017-02-03T15:35:18Z Add skeleton for the implementation of the aggregates over sliding window with processing time and time boundaries commit 2390a9d3dc15afba01185c47f61a9ea830ea5acc Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-06T10:33:57Z committing changes with stub modifications before chekout proctime branch commit eaf4e92784dab01b17004390968ca4b1fe7c4bea Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-06T13:17:43Z ignore aggregation test and implemented simple proctime test commit 16ccd7f5bf019803ea8b53f09a126ec53a5a6d59 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-06T14:17:03Z Merge branch ' FLINK-5710 ' of https://github.com/huawei-flink/flink into FLINK-5653 commit 10f7bc5e2086e41ec76cbabdcd069c71a491671a Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-07T09:42:41Z committing first key selector and utils commit 31060e46f78729880c03e8cab0f92ff06faec4f0 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-07T11:16:43Z Changed ProcTime from time to timestamp commit 69289bad836a5fdace271b28a15ca0e309e50b17 Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-07T13:13:23Z Merge branch ' FLINK-5710 ' of https://github.com/huawei-flink/flink into FLINK-5654 commit 3392817045ed166df5f55d22fde34cbd98c775db Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-07T13:14:50Z Merge branch ' FLINK-5653 ' of https://github.com/huawei-flink/flink into FLINK-5654 commit d2ea0076b5e3561585c4eaea84025e50beaacf9a Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-07T09:42:41Z fixing linelength and other issues commit f29f564bb7fe7496b9f3d2f45a6b4469af559378 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-07T13:46:30Z Merge branch ' FLINK-5653 ' of https://github.com/huawei-flink/flink.git into FLINK-5653 Conflicts: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/DataStreamWindowRowAggregate.java flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/StreamGroupKeySelector.java commit ea145ecefc2be1bea71e995dbf39585e7fa44012 Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-07T13:46:53Z Merge branch ' FLINK-5653 ' of https://github.com/huawei-flink/flink into FLINK-5654 commit 30749fc34d96a00427d079f803d44ba7007e619b Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-08T17:52:18Z Added the integration of the WindowProcTime object in the convertion rules and enable the convertion commit b001ce5505833773a9143ff1f8581eb308357b6d Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-09T14:06:48Z pushing Util function to synchronize with other branch commit 41c9ed49359b2ae9ad552eb07e51c8a60c219e97 Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-09T14:07:35Z Merge branch ' FLINK-5653 ' of https://github.com/huawei-flink/flink into FLINK-5654 commit 9c0dc0b17824ec967635c93974004533db3118b9 Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-09T16:57:49Z partition verification commit cb3d7050fff0e119c1e49b7b2ad5294774041091 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-10T14:48:03Z First version of ProcTime aggregation with Max, Min and Sum. To be tested in a deeper way, as well as introducing other aggregations commit 92d96bfe7e4dfe6426e10e59237fbae3cf772b61 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-10T15:00:22Z Merge branch ' FLINK-5654 ' of https://github.com/huawei-flink/flink into FLINK-5653 Conflicts: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/WindowAggregateUtil.java commit 2d6ea54cdfe7096fcd6bbb8ae8bbdf2cd2ba2d0c Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-10T16:53:03Z fixing unused imports commit 40f5ff80bb3023ce1125094dca135bc230ab67ba Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-28T15:52:35Z Adding aggregation functions in new package commit 65f47e99f3770a4980a3ddc412a6904ca5c69c55 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-28T15:59:08Z pushing first version of windowed aggregation working on keyed stream commit abe533b82b9e96ff63d52d405530d591ff819eb7 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-28T17:23:00Z first implementation of proctime row based aggregation commit 14ff2e9cb7e3b313d64b194956e888d8a5c3cf95 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-03-01T09:04:05Z renaming test class commit d6830b3c6be7b304a7bf7fbff464c197b8dd6591 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-03-01T13:46:09Z fixing style and removing aggregate time references commit 3cf37f0222a8c39b74b46873cc5f5ac12adabebf Author: rtudoran <rtudoran@bigdata-hp3> Date: 2017-02-02T10:30:40Z Disambiguate for the OVER BY clause, which should not be treated as a RexOver expression in Logical Project commit ea0f7af0cc7bc27975b6d1a2a946c9d0bf73cb73 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-02T12:07:11Z Added return to disambiguation method for rexover
          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/3443#discussion_r103835088

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          There have been a series of classes in scala do the aggregate, org.apache.flink.table.runtime.aggregate.*, i wonder if we should create another one.

          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/3443#discussion_r103835088 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – There have been a series of classes in scala do the aggregate, org.apache.flink.table.runtime.aggregate.*, i wonder if we should create another one.
          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/3443#discussion_r103835080

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeRowAggregate.java —
          @@ -0,0 +1,160 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.calcite.plan.RelOptCluster;
          +import org.apache.calcite.plan.RelTraitSet;
          +import org.apache.calcite.rel.RelNode;
          +import org.apache.calcite.rel.RelWriter;
          +import org.apache.calcite.rel.core.Window.Group;
          +import org.apache.calcite.rel.core.Window.RexWinAggCall;
          +import org.apache.calcite.rel.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.calcite.rex.RexInputRef;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.tuple.Tuple;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.datastream.KeyedStream;
          +import org.apache.flink.table.api.StreamTableEnvironment;
          +import org.apache.flink.table.api.TableConfig;
          +import org.apache.flink.table.calcite.FlinkTypeFactory;
          +import org.apache.flink.table.plan.logical.rel.util.WindowAggregateUtil;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateRowGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateRowKeyedWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeRowAggregate extends DataStreamRelJava {
          +
          + protected LogicalWindow windowRef;
          + protected String description;
          + protected WindowAggregateUtil winUtil;
          +
          + public DataStreamProcTimeRowAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
          + RelDataType rowType, String description, LogicalWindow window)

          { + super(cluster, traitSet, input); + this.windowRef = window; + this.rowType = rowType; + this.description = description; + this.winUtil = new WindowAggregateUtil(); + }

          +
          + @Override
          + protected RelDataType deriveRowType()

          { + return super.deriveRowType(); + }

          +
          + @Override
          + public RelNode copy(RelTraitSet traitSet, java.util.List<RelNode> inputs) {
          +
          + if (inputs.size() != 1)

          { + System.err.println(this.getClass().getName() + " : Input size must be one!"); + }

          +
          + return new DataStreamProcTimeRowAggregate(getCluster(), traitSet, inputs.get(0), getRowType(), getDescription(),
          + windowRef);
          + }
          +
          + @Override
          + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv, Row ignore) {
          +
          + TableConfig config = tableEnv.getConfig();
          +
          + DataStream<Row> inputDS = ((DataStreamRel) getInput()).translateToPlan(tableEnv);
          +
          + System.out.println(inputDS);
          — End diff –

          That should be removed.

          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/3443#discussion_r103835080 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeRowAggregate.java — @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.Window.Group; +import org.apache.calcite.rel.core.Window.RexWinAggCall; +import org.apache.calcite.rel.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexInputRef; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.KeyedStream; +import org.apache.flink.table.api.StreamTableEnvironment; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.calcite.FlinkTypeFactory; +import org.apache.flink.table.plan.logical.rel.util.WindowAggregateUtil; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateRowGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateRowKeyedWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeRowAggregate extends DataStreamRelJava { + + protected LogicalWindow windowRef; + protected String description; + protected WindowAggregateUtil winUtil; + + public DataStreamProcTimeRowAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + RelDataType rowType, String description, LogicalWindow window) { + super(cluster, traitSet, input); + this.windowRef = window; + this.rowType = rowType; + this.description = description; + this.winUtil = new WindowAggregateUtil(); + } + + @Override + protected RelDataType deriveRowType() { + return super.deriveRowType(); + } + + @Override + public RelNode copy(RelTraitSet traitSet, java.util.List<RelNode> inputs) { + + if (inputs.size() != 1) { + System.err.println(this.getClass().getName() + " : Input size must be one!"); + } + + return new DataStreamProcTimeRowAggregate(getCluster(), traitSet, inputs.get(0), getRowType(), getDescription(), + windowRef); + } + + @Override + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv, Row ignore) { + + TableConfig config = tableEnv.getConfig(); + + DataStream<Row> inputDS = ((DataStreamRel) getInput()).translateToPlan(tableEnv); + + System.out.println(inputDS); — End diff – That should be removed.
          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/3443#discussion_r103835059

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/WindowAggregateUtil.java —
          @@ -0,0 +1,103 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.rel.util;
          +
          +import java.io.Serializable;
          +import java.util.List;
          +
          +import org.apache.calcite.rel.core.Window.Group;
          +import org.apache.calcite.rel.logical.LogicalWindow;
          +import org.apache.calcite.rex.RexLiteral;
          +
          +import com.google.common.collect.ImmutableList;
          +
          +public class WindowAggregateUtil implements Serializable {
          +
          + private static final long serialVersionUID = -3916551736243544540L;
          +
          + private LogicalWindow windowPointer = null;
          +
          + public WindowAggregateUtil()

          { + + }

          +
          + public WindowAggregateUtil(LogicalWindow window)

          { + this.windowPointer = window; + + }

          +
          + /**
          + * A utility function that checks whether a window is partitioned or it is a
          + * global window.
          + *
          + * @param LogicalWindow
          + * window to be checked for partitions
          + * @return true if partition keys are defined, false otherwise.
          + */
          + public boolean isStreamPartitioned(LogicalWindow window) {
          + // if it exists a group bounded by keys, the it is
          + // a partitioned window
          + for (Group group : window.groups) {
          + if (!group.keys.isEmpty())

          { + return true; + }

          + }
          +
          + return false;
          + }
          +
          + public int[] getKeysAsArray(Group group) {
          + if (group == null)

          { + return null; + }

          + return group.keys.toArray();
          + }
          +
          + /**
          + * This method returns the [[int]] lowerbound of a window when expressed
          + * with an integer e.g. ... ROWS BETWEEN [[value]] PRECEDING AND CURRENT ROW
          + *
          + * @param constants
          + * the list of constant to get the offset value
          + * @return return the value of the lowerbound if available -1 otherwise
          + */
          +
          + public int getLowerBoundary(ImmutableList<RexLiteral> constants) {
          + return ((Long)constants.get(0).getValue2()).intValue();
          — End diff –

          I think maybe we can not get the boundary just constants[0], such as
          ```
          select sum(2) over (order by rowtime() range interval '1' hour preceding) as u3 FROM OrderA
          ```
          the boundary will be constants[1],
          you can use constants.get(lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex - inputRowType.getFieldCount)

          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/3443#discussion_r103835059 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/WindowAggregateUtil.java — @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.rel.util; + +import java.io.Serializable; +import java.util.List; + +import org.apache.calcite.rel.core.Window.Group; +import org.apache.calcite.rel.logical.LogicalWindow; +import org.apache.calcite.rex.RexLiteral; + +import com.google.common.collect.ImmutableList; + +public class WindowAggregateUtil implements Serializable { + + private static final long serialVersionUID = -3916551736243544540L; + + private LogicalWindow windowPointer = null; + + public WindowAggregateUtil() { + + } + + public WindowAggregateUtil(LogicalWindow window) { + this.windowPointer = window; + + } + + /** + * A utility function that checks whether a window is partitioned or it is a + * global window. + * + * @param LogicalWindow + * window to be checked for partitions + * @return true if partition keys are defined, false otherwise. + */ + public boolean isStreamPartitioned(LogicalWindow window) { + // if it exists a group bounded by keys, the it is + // a partitioned window + for (Group group : window.groups) { + if (!group.keys.isEmpty()) { + return true; + } + } + + return false; + } + + public int[] getKeysAsArray(Group group) { + if (group == null) { + return null; + } + return group.keys.toArray(); + } + + /** + * This method returns the [ [int] ] lowerbound of a window when expressed + * with an integer e.g. ... ROWS BETWEEN [ [value] ] PRECEDING AND CURRENT ROW + * + * @param constants + * the list of constant to get the offset value + * @return return the value of the lowerbound if available -1 otherwise + */ + + public int getLowerBoundary(ImmutableList<RexLiteral> constants) { + return ((Long)constants.get(0).getValue2()).intValue(); — End diff – I think maybe we can not get the boundary just constants [0] , such as ``` select sum(2) over (order by rowtime() range interval '1' hour preceding) as u3 FROM OrderA ``` the boundary will be constants [1] , you can use constants.get(lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex - inputRowType.getFieldCount)
          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/3443#discussion_r103835113

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,278 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.types.Row;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +import java.util.ArrayList;
          +import java.util.List;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testGlobalSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testAvgAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          — End diff –

          You may add some test about select other row attribute not in partition.

          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/3443#discussion_r103835113 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.table.api.java.StreamTableEnvironment; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.types.Row; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testGlobalSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testAvgAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); — End diff – You may add some test about select other row attribute not in partition.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103865652

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/WindowAggregateUtil.java —
          @@ -0,0 +1,103 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.rel.util;
          +
          +import java.io.Serializable;
          +import java.util.List;
          +
          +import org.apache.calcite.rel.core.Window.Group;
          +import org.apache.calcite.rel.logical.LogicalWindow;
          +import org.apache.calcite.rex.RexLiteral;
          +
          +import com.google.common.collect.ImmutableList;
          +
          +public class WindowAggregateUtil implements Serializable {
          +
          + private static final long serialVersionUID = -3916551736243544540L;
          +
          + private LogicalWindow windowPointer = null;
          +
          + public WindowAggregateUtil()

          { + + }

          +
          + public WindowAggregateUtil(LogicalWindow window)

          { + this.windowPointer = window; + + }

          +
          + /**
          + * A utility function that checks whether a window is partitioned or it is a
          + * global window.
          + *
          + * @param LogicalWindow
          + * window to be checked for partitions
          + * @return true if partition keys are defined, false otherwise.
          + */
          + public boolean isStreamPartitioned(LogicalWindow window) {
          + // if it exists a group bounded by keys, the it is
          + // a partitioned window
          + for (Group group : window.groups) {
          + if (!group.keys.isEmpty())

          { + return true; + }

          + }
          +
          + return false;
          + }
          +
          + public int[] getKeysAsArray(Group group) {
          + if (group == null)

          { + return null; + }

          + return group.keys.toArray();
          + }
          +
          + /**
          + * This method returns the [[int]] lowerbound of a window when expressed
          + * with an integer e.g. ... ROWS BETWEEN [[value]] PRECEDING AND CURRENT ROW
          + *
          + * @param constants
          + * the list of constant to get the offset value
          + * @return return the value of the lowerbound if available -1 otherwise
          + */
          +
          + public int getLowerBoundary(ImmutableList<RexLiteral> constants) {
          + return ((Long)constants.get(0).getValue2()).intValue();
          — End diff –

          Thank you very much. I was indeed puzzled by this. I will fix it according to your suggestion. I even tried to get to the Calcite mailing list, but nothing like this came out.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103865652 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/logical/rel/util/WindowAggregateUtil.java — @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.rel.util; + +import java.io.Serializable; +import java.util.List; + +import org.apache.calcite.rel.core.Window.Group; +import org.apache.calcite.rel.logical.LogicalWindow; +import org.apache.calcite.rex.RexLiteral; + +import com.google.common.collect.ImmutableList; + +public class WindowAggregateUtil implements Serializable { + + private static final long serialVersionUID = -3916551736243544540L; + + private LogicalWindow windowPointer = null; + + public WindowAggregateUtil() { + + } + + public WindowAggregateUtil(LogicalWindow window) { + this.windowPointer = window; + + } + + /** + * A utility function that checks whether a window is partitioned or it is a + * global window. + * + * @param LogicalWindow + * window to be checked for partitions + * @return true if partition keys are defined, false otherwise. + */ + public boolean isStreamPartitioned(LogicalWindow window) { + // if it exists a group bounded by keys, the it is + // a partitioned window + for (Group group : window.groups) { + if (!group.keys.isEmpty()) { + return true; + } + } + + return false; + } + + public int[] getKeysAsArray(Group group) { + if (group == null) { + return null; + } + return group.keys.toArray(); + } + + /** + * This method returns the [ [int] ] lowerbound of a window when expressed + * with an integer e.g. ... ROWS BETWEEN [ [value] ] PRECEDING AND CURRENT ROW + * + * @param constants + * the list of constant to get the offset value + * @return return the value of the lowerbound if available -1 otherwise + */ + + public int getLowerBoundary(ImmutableList<RexLiteral> constants) { + return ((Long)constants.get(0).getValue2()).intValue(); — End diff – Thank you very much. I was indeed puzzled by this. I will fix it according to your suggestion. I even tried to get to the Calcite mailing list, but nothing like this came out.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103866134

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          Good point. I searched a while, but could not find something fit, then I decided to create stream specific. The idea was to have something that could be stream optimized eventually. However, I will try to reuse existing one.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103866134 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – Good point. I searched a while, but could not find something fit, then I decided to create stream specific. The idea was to have something that could be stream optimized eventually. However, I will try to reuse existing one.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103866164

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,278 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.types.Row;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +import java.util.ArrayList;
          +import java.util.List;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testGlobalSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testAvgAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          — End diff –

          good point, will do.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103866164 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.table.api.java.StreamTableEnvironment; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.types.Row; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testGlobalSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testAvgAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); — End diff – good point, will do.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103887096

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          The aggregate functions in the org.apache.flink.table.runtime.aggregate.* assume a GroupReduce, whereas I have implemented it as a flatmap. Should I switch my implementation to reduce? @fhueske what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103887096 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – The aggregate functions in the org.apache.flink.table.runtime.aggregate.* assume a GroupReduce, whereas I have implemented it as a flatmap. Should I switch my implementation to reduce? @fhueske 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/3443#discussion_r103909104

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          Actually, we are replacing the current aggregation functions by a new interface that also allows to define user-defined aggregate functions. The functions are already merged (https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala), but not used. We'll probably merge a change to activate and use them later today. This interface should be used because otherwise user-defined aggregation function won't be supported.

          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/3443#discussion_r103909104 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – Actually, we are replacing the current aggregation functions by a new interface that also allows to define user-defined aggregate functions. The functions are already merged ( https://github.com/apache/flink/blob/master/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala ), but not used. We'll probably merge a change to activate and use them later today. This interface should be used because otherwise user-defined aggregation function won't be supported.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103914685

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          Do you think this switch on aggregation could be done in another PR?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103914685 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – Do you think this switch on aggregation could be done in another PR?
          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/3443#discussion_r103918476

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          No, I think we should use the new aggregation functions for the PR. Do you think that's 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/3443#discussion_r103918476 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – No, I think we should use the new aggregation functions for the PR. Do you think that's possible?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103919690

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          ok, so I will change my code to use the AggregateFunction you pointed out

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103919690 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – ok, so I will change my code to use the AggregateFunction you pointed out
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103920777

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          I see that the AggregateFunction interface does not have a reset method if not the createAccumulator(), this means that we have to create an accumulator for every apply execution. With the reset you just assign the value to the starting point, without creating a new object all time. What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103920777 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – I see that the AggregateFunction interface does not have a reset method if not the createAccumulator(), this means that we have to create an accumulator for every apply execution. With the reset you just assign the value to the starting point, without creating a new object all time. 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/3443#discussion_r103924165

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          The interface will be extended with an retract method to "subtract" records. @shaoxuan-wang can comment on that. he is working on the UDAGGs

          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/3443#discussion_r103924165 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – The interface will be extended with an retract method to "subtract" records. @shaoxuan-wang can comment on that. he is working on the UDAGGs
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r103925068

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          Indeed, the StreamAggregator interface I defined has a reset and evict method, to support associative function aggregation. Anyway, no point in discussing it. I will push my proposal and then you see

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r103925068 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – Indeed, the StreamAggregator interface I defined has a reset and evict method, to support associative function aggregation. Anyway, no point in discussing it. I will push my proposal and then you see
          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/3443#discussion_r103926009

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          There was a discussion on the dev ML about the new aggregation interface (https://lists.apache.org/thread.html/c4e8bbcb32731e5b37ec0a561effb279034199b495fae40b1d708406@%3Cdev.flink.apache.org%3E). Would be great if you could comment there if you think something needs to be changed. 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/3443#discussion_r103926009 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – There was a discussion on the dev ML about the new aggregation interface ( https://lists.apache.org/thread.html/c4e8bbcb32731e5b37ec0a561effb279034199b495fae40b1d708406@%3Cdev.flink.apache.org%3E ). Would be great if you could comment there if you think something needs to be changed. Thanks
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user shaoxuan-wang commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3443#discussion_r104096189

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          @huawei-flink , I have replied your question in the UDAGG design doc. AggregateFunction is the base class for UDAGG. We are very cautious to add any new method into this interface. As mentioned in the UDAGG design doc, only createAccumulator, getValue, accumulate are the must to have methods for an aggregate. Merge methods is optional only useful for advanced optimization for the runtime execution plan. Retract may also be a must-have if the users are care about the correctness. I do not see why reset is necessary for aggregate. If it is helpful in your case, you can always add this method in your User(you as the user) Defined Aggregate Function. UDAGG is still on the way, but I think it should be available very soon.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shaoxuan-wang commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r104096189 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – @huawei-flink , I have replied your question in the UDAGG design doc. AggregateFunction is the base class for UDAGG. We are very cautious to add any new method into this interface. As mentioned in the UDAGG design doc, only createAccumulator, getValue, accumulate are the must to have methods for an aggregate. Merge methods is optional only useful for advanced optimization for the runtime execution plan. Retract may also be a must-have if the users are care about the correctness. I do not see why reset is necessary for aggregate. If it is helpful in your case, you can always add this method in your User(you as the user) Defined Aggregate Function. UDAGG is still on the way, but I think it should be available very soon.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r104148648

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java —
          @@ -0,0 +1,214 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs;
          +
          +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
          +
          +import org.apache.flink.api.java.summarize.aggregation.Aggregator;
          +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum;
          +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator;
          +
          +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> {
          — End diff –

          That is fine. I have pulled the new aggregation functions, and updated the PR without the reset method.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r104148648 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/DoubleSummaryAggregation.java — @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggs; + +import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO; + +import org.apache.flink.api.java.summarize.aggregation.Aggregator; +import org.apache.flink.api.java.summarize.aggregation.CompensatedSum; +import org.apache.flink.api.java.summarize.aggregation.NumericSummaryAggregator; + +public class DoubleSummaryAggregation extends NumericSummaryAggregator<Double> { — End diff – That is fine. I have pulled the new aggregation functions, and updated the PR without the reset method.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r104361048

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,278 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.types.Row;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +import java.util.ArrayList;
          +import java.util.List;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testGlobalSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testAvgAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          — End diff –

          I have added a test, of course did not modify this one.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r104361048 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.table.api.java.StreamTableEnvironment; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.types.Row; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testGlobalSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testAvgAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); — End diff – I have added a test, of course did not modify this one.
          Hide
          stefano.bortoli Stefano Bortoli added a comment -

          What is the plan for this issue?

          Show
          stefano.bortoli Stefano Bortoli added a comment - What is the plan for this issue?
          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/3443#discussion_r105046076

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala —
          @@ -46,6 +52,20 @@ class DataStreamCalcRule
          calc.getProgram,
          description)
          }
          +
          + override def matches(call: RelOptRuleCall): Boolean = {
          — End diff –

          This can be removed because the `RexOver` was removed from `Project` (and Calc`) during normalization.

          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/3443#discussion_r105046076 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala — @@ -46,6 +52,20 @@ class DataStreamCalcRule calc.getProgram, description) } + + override def matches(call: RelOptRuleCall): Boolean = { — End diff – This can be removed because the `RexOver` was removed from `Project` (and Calc`) during normalization.
          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/3443#discussion_r105045766

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala —
          @@ -165,6 +165,11 @@ object FlinkRuleSets {

          // merge and push unions rules
          UnionEliminatorRule.INSTANCE,
          +
          + // aggregations over intervals should be enabled to be translated also in
          + //queries with LogicalWindows, not only queries with LogicalCalc
          + ProjectWindowTransposeRule.INSTANCE,
          + ProjectToWindowRule.INSTANCE,
          — End diff –

          This rule has been added to the normalization phase. It can be removed here.

          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/3443#discussion_r105045766 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala — @@ -165,6 +165,11 @@ object FlinkRuleSets { // merge and push unions rules UnionEliminatorRule.INSTANCE, + + // aggregations over intervals should be enabled to be translated also in + //queries with LogicalWindows, not only queries with LogicalCalc + ProjectWindowTransposeRule.INSTANCE, + ProjectToWindowRule.INSTANCE, — End diff – This rule has been added to the normalization phase. It can be removed here.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r105162261

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala —
          @@ -165,6 +165,11 @@ object FlinkRuleSets {

          // merge and push unions rules
          UnionEliminatorRule.INSTANCE,
          +
          + // aggregations over intervals should be enabled to be translated also in
          + //queries with LogicalWindows, not only queries with LogicalCalc
          + ProjectWindowTransposeRule.INSTANCE,
          + ProjectToWindowRule.INSTANCE,
          — End diff –

          ok

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r105162261 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala — @@ -165,6 +165,11 @@ object FlinkRuleSets { // merge and push unions rules UnionEliminatorRule.INSTANCE, + + // aggregations over intervals should be enabled to be translated also in + //queries with LogicalWindows, not only queries with LogicalCalc + ProjectWindowTransposeRule.INSTANCE, + ProjectToWindowRule.INSTANCE, — End diff – ok
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3443#discussion_r105162342

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala —
          @@ -46,6 +52,20 @@ class DataStreamCalcRule
          calc.getProgram,
          description)
          }
          +
          + override def matches(call: RelOptRuleCall): Boolean = {
          — End diff –

          very nice. This was a work-around, better to manage it propertly

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3443#discussion_r105162342 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala — @@ -46,6 +52,20 @@ class DataStreamCalcRule calc.getProgram, description) } + + override def matches(call: RelOptRuleCall): Boolean = { — End diff – very nice. This was a work-around, better to manage it propertly
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          Hi @fhueske about the squashing, what is the best strategy? I was thinking to just get a clean branch and merge my contribution there and then push it in one commit.

          About the scala part, is it really necessary? I am in the process of "translating" some of the utils in Java to overcome the "multiple extension" limitation.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3443 Hi @fhueske about the squashing, what is the best strategy? I was thinking to just get a clean branch and merge my contribution there and then push it in one commit. About the scala part, is it really necessary? I am in the process of "translating" some of the utils in Java to overcome the "multiple extension" limitation.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi, I commented on your other PR (#3459) about mixing Java and Scala and how to squash the commits.

          Thanks, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3443 Hi, I commented on your other PR (#3459) about mixing Java and Scala and how to squash the commits. Thanks, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske, I am closing this PR and opening a new one in Scala.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3443 @fhueske, I am closing this PR and opening a new one in Scala.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink closed the pull request at:

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

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

          GitHub user huawei-flink opened a pull request:

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

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

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [ x] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [ x ] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [ x] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

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

          $ git pull https://github.com/huawei-flink/flink FLINK-5653b

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

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


          commit 0e33399e145154b2dd68e0539b65a1baaba512bd
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-03-15T12:25:54Z

          First implementation of aggregation over procTime row bounded window

          commit 77a7eff006a1a4aaec2f785994a716bcd9c84133
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-03-15T12:25:54Z

          First implementation of aggregation over procTime row bounded window

          commit ecec5527b454ae9c1c3b037103b98660729d8958
          Author: Stefano Bortoli <s.bortoli@gmail.com>
          Date: 2017-03-15T13:03:13Z

          Merge branch 'master' of https://github.com/huawei-flink/flink into
          FLINK-5653b

          1. Conflicts:
          2. flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
          3. flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala

          Show
          githubbot ASF GitHub Bot added a comment - GitHub user huawei-flink opened a pull request: https://github.com/apache/flink/pull/3547 FLINK-5653 Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [ x] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [ x ] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [ x] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed You can merge this pull request into a Git repository by running: $ git pull https://github.com/huawei-flink/flink FLINK-5653 b Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3547.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 #3547 commit 0e33399e145154b2dd68e0539b65a1baaba512bd Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-03-15T12:25:54Z First implementation of aggregation over procTime row bounded window commit 77a7eff006a1a4aaec2f785994a716bcd9c84133 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-03-15T12:25:54Z First implementation of aggregation over procTime row bounded window commit ecec5527b454ae9c1c3b037103b98660729d8958 Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-03-15T13:03:13Z Merge branch 'master' of https://github.com/huawei-flink/flink into FLINK-5653 b Conflicts: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Thanks for the new PR @huawei-flink. I'll have a look soon.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3547 Thanks for the new PR @huawei-flink. I'll have a look soon.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106193738

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,38 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +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.calcite.rel.

          { RelNode, RelWriter, SingleRel }

          import org.apache.flink.api.java.typeutils.RowTypeInfo
          import org.apache.flink.streaming.api.datastream.DataStream
          -import org.apache.flink.table.api.

          {StreamTableEnvironment, TableException}

          +import org.apache.flink.table.api.

          { StreamTableEnvironment, TableException }

          import org.apache.flink.table.calcite.FlinkTypeFactory
          import org.apache.flink.table.runtime.aggregate._
          import org.apache.flink.table.plan.nodes.OverAggregate
          import org.apache.flink.types.Row
          import org.apache.calcite.rel.core.Window
          import org.apache.calcite.rel.core.Window.Group
          -import java.util.

          {List => JList}

          +import java.util.

          { List => JList }

          import org.apache.flink.table.functions.

          {ProcTimeType, RowTimeType}

          import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
          +import org.apache.calcite.sql.`type`.BasicSqlType
          +import org.apache.flink.streaming.api.functions.windowing.WindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
          +import org.apache.flink.api.java.tuple.Tuple

          class DataStreamOverAggregate(

          • logicWindow: Window,
          • cluster: RelOptCluster,
          • traitSet: RelTraitSet,
          • inputNode: RelNode,
          • rowRelDataType: RelDataType,
          • inputType: RelDataType)
          • extends SingleRel(cluster, traitSet, inputNode)
          • with OverAggregate
          • with DataStreamRel {
            + logicWindow: Window,
            + cluster: RelOptCluster,
            + traitSet: RelTraitSet,
            + inputNode: RelNode,
            + rowRelDataType: RelDataType,
            + inputType: RelDataType)
            + extends SingleRel(cluster, traitSet, inputNode)
              • End diff –

          Indent 2 space.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106193738 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,38 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +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.calcite.rel. { RelNode, RelWriter, SingleRel } import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api. {StreamTableEnvironment, TableException} +import org.apache.flink.table.api. { StreamTableEnvironment, TableException } import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.plan.nodes.OverAggregate import org.apache.flink.types.Row import org.apache.calcite.rel.core.Window import org.apache.calcite.rel.core.Window.Group -import java.util. {List => JList} +import java.util. { List => JList } import org.apache.flink.table.functions. {ProcTimeType, RowTimeType} import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair +import org.apache.calcite.sql.`type`.BasicSqlType +import org.apache.flink.streaming.api.functions.windowing.WindowFunction +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow +import org.apache.flink.api.java.tuple.Tuple class DataStreamOverAggregate( logicWindow: Window, cluster: RelOptCluster, traitSet: RelTraitSet, inputNode: RelNode, rowRelDataType: RelDataType, inputType: RelDataType) extends SingleRel(cluster, traitSet, inputNode) with OverAggregate with DataStreamRel { + logicWindow: Window, + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputNode: RelNode, + rowRelDataType: RelDataType, + inputType: RelDataType) + extends SingleRel(cluster, traitSet, inputNode) End diff – Indent 2 space.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106197313

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          I suggest use `ProcessFunction` to implement this feature. So that you can manage data flexibly and can use incremental calculations.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106197313 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – I suggest use `ProcessFunction` to implement this feature. So that you can manage data flexibly and can use incremental calculations.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106195520

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate

          import java.util

          -import org.apache.calcite.rel.`type`._
          +import scala.collection.JavaConversions.asScalaBuffer
          +import scala.collection.mutable.ArrayBuffer
          +
          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.`type`.RelDataType
          import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rex.RexInputRef
          +import org.apache.calcite.rex.RexLiteral
          +import org.apache.calcite.rex.RexWindowBound
          +import org.apache.calcite.sql.SqlAggFunction
          +import org.apache.calcite.sql.SqlKind
          import org.apache.calcite.sql.`type`.SqlTypeName
          -import org.apache.calcite.sql.`type`.SqlTypeName._
          -import org.apache.calcite.sql.fun._
          -import org.apache.calcite.sql.

          {SqlAggFunction, SqlKind}

          -import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN
          +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL
          +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE
          +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT
          +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER
          +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT
          — End diff –

          Can you remove those unused import ?

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106195520 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate import java.util -import org.apache.calcite.rel.`type`._ +import scala.collection.JavaConversions.asScalaBuffer +import scala.collection.mutable.ArrayBuffer + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rex.RexInputRef +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.rex.RexWindowBound +import org.apache.calcite.sql.SqlAggFunction +import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun._ -import org.apache.calcite.sql. {SqlAggFunction, SqlKind} -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT — End diff – Can you remove those unused import ?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106193639

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,38 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +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.calcite.rel.

          { RelNode, RelWriter, SingleRel }

          import org.apache.flink.api.java.typeutils.RowTypeInfo
          import org.apache.flink.streaming.api.datastream.DataStream
          -import org.apache.flink.table.api.

          {StreamTableEnvironment, TableException}

          +import org.apache.flink.table.api.

          { StreamTableEnvironment, TableException }

          import org.apache.flink.table.calcite.FlinkTypeFactory
          import org.apache.flink.table.runtime.aggregate._
          import org.apache.flink.table.plan.nodes.OverAggregate
          import org.apache.flink.types.Row
          import org.apache.calcite.rel.core.Window
          import org.apache.calcite.rel.core.Window.Group
          -import java.util.

          {List => JList}

          +import java.util.

          { List => JList }

          import org.apache.flink.table.functions.

          {ProcTimeType, RowTimeType}

          import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
          +import org.apache.calcite.sql.`type`.BasicSqlType
          +import org.apache.flink.streaming.api.functions.windowing.WindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
          +import org.apache.flink.api.java.tuple.Tuple

          class DataStreamOverAggregate(

          • logicWindow: Window,
          • cluster: RelOptCluster,
          • traitSet: RelTraitSet,
          • inputNode: RelNode,
          • rowRelDataType: RelDataType,
          • inputType: RelDataType)
          • extends SingleRel(cluster, traitSet, inputNode)
          • with OverAggregate
          • with DataStreamRel {
            + logicWindow: Window,
              • End diff –

          Indent 4 space.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106193639 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,38 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +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.calcite.rel. { RelNode, RelWriter, SingleRel } import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api. {StreamTableEnvironment, TableException} +import org.apache.flink.table.api. { StreamTableEnvironment, TableException } import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.plan.nodes.OverAggregate import org.apache.flink.types.Row import org.apache.calcite.rel.core.Window import org.apache.calcite.rel.core.Window.Group -import java.util. {List => JList} +import java.util. { List => JList } import org.apache.flink.table.functions. {ProcTimeType, RowTimeType} import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair +import org.apache.calcite.sql.`type`.BasicSqlType +import org.apache.flink.streaming.api.functions.windowing.WindowFunction +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow +import org.apache.flink.api.java.tuple.Tuple class DataStreamOverAggregate( logicWindow: Window, cluster: RelOptCluster, traitSet: RelTraitSet, inputNode: RelNode, rowRelDataType: RelDataType, inputType: RelDataType) extends SingleRel(cluster, traitSet, inputNode) with OverAggregate with DataStreamRel { + logicWindow: Window, End diff – Indent 4 space.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106193220

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala —
          @@ -18,8 +18,6 @@

          package org.apache.flink.table.plan.logical.rel

          -import java.util
          — End diff –

          I think import package name is good way. 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/3547#discussion_r106193220 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala — @@ -18,8 +18,6 @@ package org.apache.flink.table.plan.logical.rel -import java.util — End diff – I think import package name is good way. 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/3547#discussion_r106192954

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala —
          @@ -37,8 +35,8 @@ class LogicalWindowAggregate(
          child: RelNode,
          indicator: Boolean,
          groupSet: ImmutableBitSet,

          • groupSets: util.List[ImmutableBitSet],
          • aggCalls: util.List[AggregateCall])
            + groupSets: java.util.List[ImmutableBitSet],
              • End diff –

          Can we keep the original way? `util.List[ImmutableBitSet]`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106192954 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala — @@ -37,8 +35,8 @@ class LogicalWindowAggregate( child: RelNode, indicator: Boolean, groupSet: ImmutableBitSet, groupSets: util.List [ImmutableBitSet] , aggCalls: util.List [AggregateCall] ) + groupSets: java.util.List [ImmutableBitSet] , End diff – Can we keep the original way? `util.List [ImmutableBitSet] `
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106195116

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

          +def createBoundedAndCurrentRowProcessingTimeOverWindow(
          + inputDS: DataStream[Row]): DataStream[Row] = {
          +
          + val overWindow: Group = logicWindow.groups.get(0)
          + val partitionKeys: Array[Int] = overWindow.keys.toArray
          + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + // partitioned aggregation
          + if (partitionKeys.nonEmpty)

          { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound, 1).apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

          // global non-partitioned aggregation
          + else {
          + throw TableException(
          — End diff –

          Can you supported not-partitioned case in this JIRA.

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106195116 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -159,6 +168,42 @@ class DataStreamOverAggregate( result } +def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound, 1).apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + throw TableException( — End diff – Can you supported not-partitioned case in this JIRA.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user sunjincheng121 commented on the issue:

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

          Hi @fhueske I'am sorry, I have not seen your comments until I passed over this PR. If I have any mistakes, please correct it.
          Best,
          SunJIncheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on the issue: https://github.com/apache/flink/pull/3547 Hi @fhueske I'am sorry, I have not seen your comments until I passed over this PR. If I have any mistakes, please correct it. Best, SunJIncheng
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          No worries @sunjincheng121!
          Everybody is welcome and encouraged to review pull requests.
          Thanks for doing this!

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3547 No worries @sunjincheng121! Everybody is welcome and encouraged to review pull requests. Thanks for doing this!
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106219673

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          @sunjincheng121 thanks for the suggestion. I decided to use Window because it is convenient in the row bounded case. Within the window I apply the incremental aggregation in the same way.

          It is not clear to me what are the flexibility advantages in this specific case. Can you be more explicit?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106219673 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – @sunjincheng121 thanks for the suggestion. I decided to use Window because it is convenient in the row bounded case. Within the window I apply the incremental aggregation in the same way. It is not clear to me what are the flexibility advantages in this specific case. Can you be more explicit?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106219814

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate

          import java.util

          -import org.apache.calcite.rel.`type`._
          +import scala.collection.JavaConversions.asScalaBuffer
          +import scala.collection.mutable.ArrayBuffer
          +
          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.`type`.RelDataType
          import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rex.RexInputRef
          +import org.apache.calcite.rex.RexLiteral
          +import org.apache.calcite.rex.RexWindowBound
          +import org.apache.calcite.sql.SqlAggFunction
          +import org.apache.calcite.sql.SqlKind
          import org.apache.calcite.sql.`type`.SqlTypeName
          -import org.apache.calcite.sql.`type`.SqlTypeName._
          -import org.apache.calcite.sql.fun._
          -import org.apache.calcite.sql.

          {SqlAggFunction, SqlKind}

          -import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN
          +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL
          +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE
          +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT
          +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER
          +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT
          — End diff –

          sure

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106219814 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate import java.util -import org.apache.calcite.rel.`type`._ +import scala.collection.JavaConversions.asScalaBuffer +import scala.collection.mutable.ArrayBuffer + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rex.RexInputRef +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.rex.RexWindowBound +import org.apache.calcite.sql.SqlAggFunction +import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun._ -import org.apache.calcite.sql. {SqlAggFunction, SqlKind} -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT — End diff – sure
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106220189

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

          +def createBoundedAndCurrentRowProcessingTimeOverWindow(
          + inputDS: DataStream[Row]): DataStream[Row] = {
          +
          + val overWindow: Group = logicWindow.groups.get(0)
          + val partitionKeys: Array[Int] = overWindow.keys.toArray
          + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + // partitioned aggregation
          + if (partitionKeys.nonEmpty)

          { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound, 1).apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

          // global non-partitioned aggregation
          + else {
          + throw TableException(
          — End diff –

          If needed, I can do it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106220189 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -159,6 +168,42 @@ class DataStreamOverAggregate( result } +def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound, 1).apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + throw TableException( — End diff – If needed, I can do it.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106221708

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala —
          @@ -37,8 +35,8 @@ class LogicalWindowAggregate(
          child: RelNode,
          indicator: Boolean,
          groupSet: ImmutableBitSet,

          • groupSets: util.List[ImmutableBitSet],
          • aggCalls: util.List[AggregateCall])
            + groupSets: java.util.List[ImmutableBitSet],
              • End diff –

          for some reason it does not build in eclipse in the original way, and honestly I struggle to understand "the half package name" usage in Scala. Is there any practical reason for that? and does it break some convention to use the complete package name? I am asking with honest curiosity, and no polemical intention. Thanks for clarifying.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106221708 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala — @@ -37,8 +35,8 @@ class LogicalWindowAggregate( child: RelNode, indicator: Boolean, groupSet: ImmutableBitSet, groupSets: util.List [ImmutableBitSet] , aggCalls: util.List [AggregateCall] ) + groupSets: java.util.List [ImmutableBitSet] , End diff – for some reason it does not build in eclipse in the original way, and honestly I struggle to understand "the half package name" usage in Scala. Is there any practical reason for that? and does it break some convention to use the complete package name? I am asking with honest curiosity, and no polemical intention. Thanks for clarifying.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106222135

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala —
          @@ -18,8 +18,6 @@

          package org.apache.flink.table.plan.logical.rel

          -import java.util
          — End diff –

          In principle I agree, but I it caused some building problem. Is there a practical reason for not importing the List class directly rather than creating "half references"?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106222135 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala — @@ -18,8 +18,6 @@ package org.apache.flink.table.plan.logical.rel -import java.util — End diff – In principle I agree, but I it caused some building problem. Is there a practical reason for not importing the List class directly rather than creating "half references"?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106238200

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate

          import java.util

          -import org.apache.calcite.rel.`type`._
          +import scala.collection.JavaConversions.asScalaBuffer
          +import scala.collection.mutable.ArrayBuffer
          +
          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.`type`.RelDataType
          import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rex.RexInputRef
          +import org.apache.calcite.rex.RexLiteral
          +import org.apache.calcite.rex.RexWindowBound
          +import org.apache.calcite.sql.SqlAggFunction
          +import org.apache.calcite.sql.SqlKind
          import org.apache.calcite.sql.`type`.SqlTypeName
          -import org.apache.calcite.sql.`type`.SqlTypeName._
          -import org.apache.calcite.sql.fun._
          -import org.apache.calcite.sql.

          {SqlAggFunction, SqlKind}

          -import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN
          +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL
          +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE
          +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT
          +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER
          +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT
          — End diff –

          after an inspection, I realized that the imports you mentioned are used. I think there is no unused import at this moment. Am I missing something?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106238200 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate import java.util -import org.apache.calcite.rel.`type`._ +import scala.collection.JavaConversions.asScalaBuffer +import scala.collection.mutable.ArrayBuffer + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rex.RexInputRef +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.rex.RexWindowBound +import org.apache.calcite.sql.SqlAggFunction +import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun._ -import org.apache.calcite.sql. {SqlAggFunction, SqlKind} -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT — End diff – after an inspection, I realized that the imports you mentioned are used. I think there is no unused import at this moment. Am I missing something?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106335062

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala —
          @@ -18,8 +18,6 @@

          package org.apache.flink.table.plan.logical.rel

          -import java.util
          — End diff –

          Your current package is `org.apache.flink.table.plan.logical.rel`, in this package there is a` org.apache.flink.table.plan.logical.rel.util`, this and `util .List` package conflicts. So you can try the following 2 way solution:
          solution1:
          ```
          import java.util.List
          groupSets: List[ImmutableBitSet],
          aggCalls: List[AggregateCall])
          ```
          solution2:
          ```
          import java.

          {util => JUtil}

          groupSets: JUtil.List[ImmutableBitSet],
          aggCalls: JUtil.List[AggregateCall])
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106335062 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala — @@ -18,8 +18,6 @@ package org.apache.flink.table.plan.logical.rel -import java.util — End diff – Your current package is `org.apache.flink.table.plan.logical.rel`, in this package there is a` org.apache.flink.table.plan.logical.rel.util`, this and `util .List` package conflicts. So you can try the following 2 way solution: solution1: ``` import java.util.List groupSets: List [ImmutableBitSet] , aggCalls: List [AggregateCall] ) ``` solution2: ``` import java. {util => JUtil} groupSets: JUtil.List [ImmutableBitSet] , aggCalls: JUtil.List [AggregateCall] ) ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106336720

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate

          import java.util

          -import org.apache.calcite.rel.`type`._
          +import scala.collection.JavaConversions.asScalaBuffer
          +import scala.collection.mutable.ArrayBuffer
          +
          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.`type`.RelDataType
          import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rex.RexInputRef
          +import org.apache.calcite.rex.RexLiteral
          +import org.apache.calcite.rex.RexWindowBound
          +import org.apache.calcite.sql.SqlAggFunction
          +import org.apache.calcite.sql.SqlKind
          import org.apache.calcite.sql.`type`.SqlTypeName
          -import org.apache.calcite.sql.`type`.SqlTypeName._
          -import org.apache.calcite.sql.fun._
          -import org.apache.calcite.sql.

          {SqlAggFunction, SqlKind}

          -import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN
          +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL
          +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE
          +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT
          +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER
          +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT
          — End diff –

          Yes, I had clone your code.May be in this case,we can use `._` ,
          e.g.: `import org.apache.flink.table.functions.aggfunctions._` In this way, we can reduce 53 line to 1 line. 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/3547#discussion_r106336720 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate import java.util -import org.apache.calcite.rel.`type`._ +import scala.collection.JavaConversions.asScalaBuffer +import scala.collection.mutable.ArrayBuffer + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rex.RexInputRef +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.rex.RexWindowBound +import org.apache.calcite.sql.SqlAggFunction +import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun._ -import org.apache.calcite.sql. {SqlAggFunction, SqlKind} -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT — End diff – Yes, I had clone your code.May be in this case,we can use `._` , e.g.: `import org.apache.flink.table.functions.aggfunctions._` In this way, we can reduce 53 line to 1 line. 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/3547#discussion_r106342776

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0");
          + expected.add("2,1");
          + expected.add("2,3");
          + expected.add("3,3");
          + expected.add("3,7");
          + expected.add("3,9");
          + expected.add("4,6");
          + expected.add("4,13");
          + expected.add("4,15");
          + expected.add("4,17");
          + expected.add("5,10");
          + expected.add("5,21");
          + expected.add("5,23");
          + expected.add("5,26");
          + expected.add("5,27");
          — End diff –

          This result does not conform to the semantics of over. that is, in this sql,we must be got the result as follow:
          ```
          1,0
          2,1
          2,3
          3,3
          3,7
          3,12
          4,6
          4,13
          4,21
          4,24
          5,10
          5,21
          5,33
          5,37
          5,39
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106342776 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); — End diff – This result does not conform to the semantics of over. that is, in this sql,we must be got the result as follow: ``` 1,0 2,1 2,3 3,3 3,7 3,12 4,6 4,13 4,21 4,24 5,10 5,21 5,33 5,37 5,39 ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106343147

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,76 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput())
            +
            + val result: DataStream[Row] =
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty)

            { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            // global non-partitioned aggregation
            + else {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction(
            + namedAggregates,
            + inputType)
            +
            + inputDS
            + .countWindowAll(lowerbound,1)

              • End diff –

          `lowerbound -> (lowerbound+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/3547#discussion_r106343147 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,76 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates, + inputType) + + inputDS + .countWindowAll(lowerbound,1) End diff – `lowerbound -> (lowerbound+1)`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106343179

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,76 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput())
            +
            + val result: DataStream[Row] =
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction(
            + namedAggregates,
            + inputType)
            + inputDS
            + .keyBy(partitionKeys: _*)
            + .countWindow(lowerbound,1)

              • End diff –

          `lowerbound -> (lowerbound+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/3547#discussion_r106343179 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,76 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) End diff – `lowerbound -> (lowerbound+1)`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106344518

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          I glad to explain this. IMO. There are several advantages to using `ProcessFunction`:
          1. OverWindow implementation code consistent, such as: FLINK-5803, FLINK-5804, FLINK-5658 (See @fhueske propose the design in #3386 )
          2. Because using `ProcessFunction` can manage their own state, the management window of the data collection, you can customize the trigger window, so in the future when the `FOLLOWING` supported we can be very convenient support.
          3. For incremental aggregation, window mechanism must increase the implementation of `reduceFunction`, `ProcessFunction` in the management of the state at the same time will be very natural support.
          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/3547#discussion_r106344518 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – I glad to explain this. IMO. There are several advantages to using `ProcessFunction`: 1. OverWindow implementation code consistent, such as: FLINK-5803 , FLINK-5804 , FLINK-5658 (See @fhueske propose the design in #3386 ) 2. Because using `ProcessFunction` can manage their own state, the management window of the data collection, you can customize the trigger window, so in the future when the `FOLLOWING` supported we can be very convenient support. 3. For incremental aggregation, window mechanism must increase the implementation of `reduceFunction`, `ProcessFunction` in the management of the state at the same time will be very natural support. 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/3547#discussion_r106344831

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testGlobalSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testAvgAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS avgC, e FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,1"); + expected.add("2,1,1"); + expected.add("2,1,2"); + expected.add("3,3,2"); + expected.add("3,3,2"); + expected.add("3,4,3"); + expected.add("4,6,1"); + expected.add("4,6,2"); + expected.add("4,7,1"); + expected.add("4,8,2"); + expected.add("5,10,1"); + expected.add("5,10,3"); + expected.add("5,11,3"); + expected.add("5,13,2"); + expected.add("5,13,2"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testAvgAggregatation2() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS avgC, d FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0,Hallo");
          + expected.add("2,1,Hallo Welt");
          + expected.add("2,1,Hallo Welt wie");
          + expected.add("3,3,ABC");
          + expected.add("3,3,Hallo Welt wie gehts?");
          + expected.add("3,4,BCD");
          + expected.add("4,6,CDE");
          + expected.add("4,6,DEF");
          + expected.add("4,7,EFG");
          + expected.add("4,8,FGH");
          + expected.add("5,10,GHI");
          + expected.add("5,10,HIJ");
          + expected.add("5,11,IJK");
          + expected.add("5,13,JKL");
          — End diff –

          This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106344831 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testGlobalSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testAvgAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS avgC, e FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,1"); + expected.add("2,1,1"); + expected.add("2,1,2"); + expected.add("3,3,2"); + expected.add("3,3,2"); + expected.add("3,4,3"); + expected.add("4,6,1"); + expected.add("4,6,2"); + expected.add("4,7,1"); + expected.add("4,8,2"); + expected.add("5,10,1"); + expected.add("5,10,3"); + expected.add("5,11,3"); + expected.add("5,13,2"); + expected.add("5,13,2"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testAvgAggregatation2() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS avgC, d FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,Hallo"); + expected.add("2,1,Hallo Welt"); + expected.add("2,1,Hallo Welt wie"); + expected.add("3,3,ABC"); + expected.add("3,3,Hallo Welt wie gehts?"); + expected.add("3,4,BCD"); + expected.add("4,6,CDE"); + expected.add("4,6,DEF"); + expected.add("4,7,EFG"); + expected.add("4,8,FGH"); + expected.add("5,10,GHI"); + expected.add("5,10,HIJ"); + expected.add("5,11,IJK"); + expected.add("5,13,JKL"); — End diff – This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106344846

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testGlobalSumAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("0");
          + expected.add("1");
          + expected.add("3");
          + expected.add("5");
          + expected.add("7");
          + expected.add("9");
          + expected.add("11");
          + expected.add("13");
          + expected.add("15");
          + expected.add("17");
          + expected.add("19");
          + expected.add("21");
          + expected.add("23");
          + expected.add("26");
          + expected.add("27");
          — End diff –

          This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106344846 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testGlobalSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); — End diff – This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106344697

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, "
          + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC,"
          + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC"
          + + " FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0,0");
          + expected.add("2,1,1");
          + expected.add("2,3,1");
          + expected.add("3,3,3");
          + expected.add("3,7,3");
          + expected.add("3,9,4");
          + expected.add("4,6,6");
          + expected.add("4,13,6");
          + expected.add("4,15,7");
          + expected.add("4,17,8");
          + expected.add("5,10,10");
          + expected.add("5,21,10");
          + expected.add("5,23,11");
          + expected.add("5,26,12");
          + expected.add("5,27,13");
          — End diff –

          This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106344697 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); — End diff – This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106344891

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0");
          + expected.add("2,1");
          + expected.add("2,1");
          + expected.add("3,3");
          + expected.add("3,3");
          + expected.add("3,4");
          + expected.add("4,6");
          + expected.add("4,6");
          + expected.add("4,7");
          + expected.add("4,8");
          + expected.add("5,10");
          + expected.add("5,10");
          + expected.add("5,11");
          + expected.add("5,12");
          — End diff –

          This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106344891 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); — End diff – This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106344788

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testGlobalSumAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testAvgAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS avgC, e FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0,1");
          + expected.add("2,1,1");
          + expected.add("2,1,2");
          + expected.add("3,3,2");
          + expected.add("3,3,2");
          + expected.add("3,4,3");
          + expected.add("4,6,1");
          + expected.add("4,6,2");
          + expected.add("4,7,1");
          + expected.add("4,8,2");
          + expected.add("5,10,1");
          + expected.add("5,10,3");
          + expected.add("5,11,3");
          + expected.add("5,13,2");
          + expected.add("5,13,2");
          — End diff –

          This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106344788 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, " + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC," + + "MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC" + + " FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,0"); + expected.add("2,1,1"); + expected.add("2,3,1"); + expected.add("3,3,3"); + expected.add("3,7,3"); + expected.add("3,9,4"); + expected.add("4,6,6"); + expected.add("4,13,6"); + expected.add("4,15,7"); + expected.add("4,17,8"); + expected.add("5,10,10"); + expected.add("5,21,10"); + expected.add("5,23,11"); + expected.add("5,26,12"); + expected.add("5,27,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testGlobalSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT SUM(c) OVER (ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("0"); + expected.add("1"); + expected.add("3"); + expected.add("5"); + expected.add("7"); + expected.add("9"); + expected.add("11"); + expected.add("13"); + expected.add("15"); + expected.add("17"); + expected.add("19"); + expected.add("21"); + expected.add("23"); + expected.add("26"); + expected.add("27"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testAvgAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS avgC, e FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0,1"); + expected.add("2,1,1"); + expected.add("2,1,2"); + expected.add("3,3,2"); + expected.add("3,3,2"); + expected.add("3,4,3"); + expected.add("4,6,1"); + expected.add("4,6,2"); + expected.add("4,7,1"); + expected.add("4,8,2"); + expected.add("5,10,1"); + expected.add("5,10,3"); + expected.add("5,11,3"); + expected.add("5,13,2"); + expected.add("5,13,2"); — End diff – This result incorrect.The reason for the error I left the comments in `DataStreamOverAggregate.scala`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106360688

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala —
          @@ -18,8 +18,6 @@

          package org.apache.flink.table.plan.logical.rel

          -import java.util
          — End diff –

          Thanks, both solutions look better than the half package import. I will apply one of those.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106360688 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala — @@ -18,8 +18,6 @@ package org.apache.flink.table.plan.logical.rel -import java.util — End diff – Thanks, both solutions look better than the half package import. I will apply one of those.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106361152

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate

          import java.util

          -import org.apache.calcite.rel.`type`._
          +import scala.collection.JavaConversions.asScalaBuffer
          +import scala.collection.mutable.ArrayBuffer
          +
          +import org.apache.calcite.rel.RelNode
          +import org.apache.calcite.rel.`type`.RelDataType
          import org.apache.calcite.rel.core.AggregateCall
          +import org.apache.calcite.rex.RexInputRef
          +import org.apache.calcite.rex.RexLiteral
          +import org.apache.calcite.rex.RexWindowBound
          +import org.apache.calcite.sql.SqlAggFunction
          +import org.apache.calcite.sql.SqlKind
          import org.apache.calcite.sql.`type`.SqlTypeName
          -import org.apache.calcite.sql.`type`.SqlTypeName._
          -import org.apache.calcite.sql.fun._
          -import org.apache.calcite.sql.

          {SqlAggFunction, SqlKind}

          -import org.apache.flink.streaming.api.functions.ProcessFunction
          +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN
          +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL
          +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE
          +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT
          +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER
          +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT
          +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT
          — End diff –

          I see your point, although I thought that wildcard import was not a best practice. It seems that the java and scala implementation are following different conventions. I have no problems with it in principle.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106361152 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -19,33 +19,124 @@ package org.apache.flink.table.runtime.aggregate import java.util -import org.apache.calcite.rel.`type`._ +import scala.collection.JavaConversions.asScalaBuffer +import scala.collection.mutable.ArrayBuffer + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rex.RexInputRef +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.rex.RexWindowBound +import org.apache.calcite.sql.SqlAggFunction +import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun._ -import org.apache.calcite.sql. {SqlAggFunction, SqlKind} -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.calcite.sql.`type`.SqlTypeName.BIGINT +import org.apache.calcite.sql.`type`.SqlTypeName.BOOLEAN +import org.apache.calcite.sql.`type`.SqlTypeName.DECIMAL +import org.apache.calcite.sql.`type`.SqlTypeName.DOUBLE +import org.apache.calcite.sql.`type`.SqlTypeName.FLOAT +import org.apache.calcite.sql.`type`.SqlTypeName.INTEGER +import org.apache.calcite.sql.`type`.SqlTypeName.SMALLINT +import org.apache.calcite.sql.`type`.SqlTypeName.TINYINT — End diff – I see your point, although I thought that wildcard import was not a best practice. It seems that the java and scala implementation are following different conventions. I have no problems with it in principle.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106366744

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          Thanks a lot for the clarification. I am really willing to do it right, but at the same time I need to understand. So, please be patient.

          When we started discussing the issue with @fhueske (https://issues.apache.org/jira/browse/FLINK-5654?filter=-1) there was a decision to use window, not process function.
          Code consistency is pretty much the same, just extening a different interface. I understand that ProcessFunction can manage its state, but window checkpointing should replay all events in case of failure, so we would have consistent processing even without managing this level of granularity in the state. With procTime semantic, we can neglect retraction, and window can anyway customize triggering function.

          I don't understand the third point.

          The main argument I see for this specific case is that ProcessFunction supports granular state management. Besides the alleged code consistency.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106366744 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – Thanks a lot for the clarification. I am really willing to do it right, but at the same time I need to understand. So, please be patient. When we started discussing the issue with @fhueske ( https://issues.apache.org/jira/browse/FLINK-5654?filter=-1 ) there was a decision to use window, not process function. Code consistency is pretty much the same, just extening a different interface. I understand that ProcessFunction can manage its state, but window checkpointing should replay all events in case of failure, so we would have consistent processing even without managing this level of granularity in the state. With procTime semantic, we can neglect retraction, and window can anyway customize triggering function. I don't understand the third point. The main argument I see for this specific case is that ProcessFunction supports granular state management. Besides the alleged code consistency.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106367130

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,76 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput())
            +
            + val result: DataStream[Row] =
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty)

            { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            // global non-partitioned aggregation
            + else {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction(
            + namedAggregates,
            + inputType)
            +
            + inputDS
            + .countWindowAll(lowerbound,1)

              • End diff –

          So, the semantic of between 2 rows and current row does not include the current row and I should count 3 elements?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106367130 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,76 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates, + inputType) + + inputDS + .countWindowAll(lowerbound,1) End diff – So, the semantic of between 2 rows and current row does not include the current row and I should count 3 elements?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106367506

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          — End diff –

          Why should also the test be implemented in scala?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106367506 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { — End diff – Why should also the test be implemented in scala?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106368103

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          Hi,@huawei-flink about the discussing I think we need @fhueske 's opinion.
          Best,
          SunJincheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106368103 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – Hi,@huawei-flink about the discussing I think we need @fhueske 's opinion. Best, SunJincheng
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106369028

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0");
          + expected.add("2,1");
          + expected.add("2,3");
          + expected.add("3,3");
          + expected.add("3,7");
          + expected.add("3,9");
          + expected.add("4,6");
          + expected.add("4,13");
          + expected.add("4,15");
          + expected.add("4,17");
          + expected.add("5,10");
          + expected.add("5,21");
          + expected.add("5,23");
          + expected.add("5,26");
          + expected.add("5,27");
          — End diff –

          Besides the fix you suggested, which is trivial (and thanks! ), it sounds a little odd to define a range with excluding boundaries. could you please point out the document in which this is discussed?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106369028 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); — End diff – Besides the fix you suggested, which is trivial (and thanks! ), it sounds a little odd to define a range with excluding boundaries. could you please point out the document in which this is discussed?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106452550

          — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java —
          @@ -0,0 +1,317 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList;
          +import java.util.List;
          +
          +import org.apache.flink.api.java.tuple.Tuple5;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
          +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
          +import org.apache.flink.table.api.Table;
          +import org.apache.flink.table.api.TableEnvironment;
          +import org.apache.flink.table.api.java.StreamTableEnvironment;
          +import org.apache.flink.table.api.java.stream.utils.StreamTestData;
          +import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
          +import org.apache.flink.types.Row;
          +import org.junit.Ignore;
          +import org.junit.Test;
          +
          +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase {
          +
          +
          + @Test
          + public void testMaxAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testMinAggregatation() throws Exception

          { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + }

          +
          + @Test
          + public void testSumAggregatation() throws Exception {
          + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
          + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
          + StreamITCase.clear();
          +
          + env.setParallelism(1);
          +
          + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
          + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e");
          + tableEnv.registerTable("MyTable", in);
          +
          + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable";
          + Table result = tableEnv.sql(sqlQuery);
          +
          + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
          + resultSet.addSink(new StreamITCase.StringSink());
          + env.execute();
          +
          + List<String> expected = new ArrayList<>();
          + expected.add("1,0");
          + expected.add("2,1");
          + expected.add("2,3");
          + expected.add("3,3");
          + expected.add("3,7");
          + expected.add("3,9");
          + expected.add("4,6");
          + expected.add("4,13");
          + expected.add("4,15");
          + expected.add("4,17");
          + expected.add("5,10");
          + expected.add("5,21");
          + expected.add("5,23");
          + expected.add("5,26");
          + expected.add("5,27");
          — End diff –

          It makes sense anyway, I fixed according to your suggestion.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106452550 — Diff: flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/ProcTimeRowStreamAggregationSqlITCase.java — @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.api.java.stream.utils.StreamTestData; +import org.apache.flink.table.api.scala.stream.utils.StreamITCase; +import org.apache.flink.types.Row; +import org.junit.Ignore; +import org.junit.Test; + +public class ProcTimeRowStreamAggregationSqlITCase extends StreamingMultipleProgramsTestBase { + + + @Test + public void testMaxAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MAX(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,2"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("3,5"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("4,9"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,14"); + expected.add("5,14"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testMinAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, MIN(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS maxC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,1"); + expected.add("3,3"); + expected.add("3,3"); + expected.add("3,4"); + expected.add("4,6"); + expected.add("4,6"); + expected.add("4,7"); + expected.add("4,8"); + expected.add("5,10"); + expected.add("5,10"); + expected.add("5,11"); + expected.add("5,12"); + expected.add("5,13"); + + StreamITCase.compareWithList(expected); + } + + @Test + public void testSumAggregatation() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + StreamITCase.clear(); + + env.setParallelism(1); + + DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env); + Table in = tableEnv.fromDataStream(ds, "a,b,c,d,e"); + tableEnv.registerTable("MyTable", in); + + String sqlQuery = "SELECT a, SUM(c) OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC FROM MyTable"; + Table result = tableEnv.sql(sqlQuery); + + DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class); + resultSet.addSink(new StreamITCase.StringSink()); + env.execute(); + + List<String> expected = new ArrayList<>(); + expected.add("1,0"); + expected.add("2,1"); + expected.add("2,3"); + expected.add("3,3"); + expected.add("3,7"); + expected.add("3,9"); + expected.add("4,6"); + expected.add("4,13"); + expected.add("4,15"); + expected.add("4,17"); + expected.add("5,10"); + expected.add("5,21"); + expected.add("5,23"); + expected.add("5,26"); + expected.add("5,27"); — End diff – It makes sense anyway, I fixed according to your suggestion.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106465657

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          I just thought about a case where one wants to do a COUNT DISTINCT type of aggregation. How does a processFunction work for that?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106465657 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – I just thought about a case where one wants to do a COUNT DISTINCT type of aggregation. How does a processFunction work for that?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106488388

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          As @huawei-flink pointed out, I agreed to use a WindowFunction for the bounded OVER ROW case in our previous discussions. Although it would not be consistent with the other OVER windows, this would be an easier solution, IMO.

          However, I think we have to use a `ProcessFunction` for correctness reasons. An OVER aggregation must emit one row for each input row. If we have an `OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 3 PRECEDING AND CURRENT ROW)` and implement this with a `SlidingCountWindow(3, 1)` we will lose the two rows because the window function is called the first time, when three elements have arrived.

          Regarding the aggregation of COUNT DISTINCT, we could implement a custom aggregation function, similar to the retractable min or max aggregation function.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106488388 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – As @huawei-flink pointed out, I agreed to use a WindowFunction for the bounded OVER ROW case in our previous discussions. Although it would not be consistent with the other OVER windows, this would be an easier solution, IMO. However, I think we have to use a `ProcessFunction` for correctness reasons. An OVER aggregation must emit one row for each input row. If we have an `OVER (PARTITION BY a ORDER BY procTime() ROWS BETWEEN 3 PRECEDING AND CURRENT ROW)` and implement this with a `SlidingCountWindow(3, 1)` we will lose the two rows because the window function is called the first time, when three elements have arrived. Regarding the aggregation of COUNT DISTINCT, we could implement a custom aggregation function, similar to the retractable min or max aggregation function.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106525346

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

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + private[flink] def CreateBoundedProcessingOverGlobalWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): AllWindowFunction[Row,Row,GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverAllWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + def getLowerBoundary(
          + constants: ImmutableList[RexLiteral],
          + lowerBound: RexWindowBound,
          + input: RelNode):Int = {
          — End diff –

          `input: ReoNode): Int = {` +space

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106525346 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + private [flink] def CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): AllWindowFunction [Row,Row,GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverAllWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode):Int = { — End diff – `input: ReoNode): Int = {` +space
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106517628

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,77 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + // window size is lowerbound +1 to comply with over semantics
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput()) + 1
            +
            + val result: DataStream[Row] =
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction(
            + namedAggregates,
            + inputType)
            + inputDS
            + .keyBy(partitionKeys: _*)
            + .countWindow(lowerbound,1)

              • End diff –

          `.countWindow(lowerbound, 1)` +space

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106517628 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,77 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + 1 + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) End diff – `.countWindow(lowerbound, 1)` +space
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106519205

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,76 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            + result
            + }
            +
            + def createBoundedAndCurrentRowProcessingTimeOverWindow(
            + inputDS: DataStream[Row]): DataStream[Row] = {
            +
            + val overWindow: Group = logicWindow.groups.get(0)
            + val partitionKeys: Array[Int] = overWindow.keys.toArray
            + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
            +
            + // get the output types
            + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
            +
            + val lowerbound: Int = AggregateUtil.getLowerBoundary(
            + logicWindow.constants,
            + overWindow.lowerBound,
            + getInput())
            +
            + val result: DataStream[Row] =
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty)

            { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            // global non-partitioned aggregation
            + else {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction(
            + namedAggregates,
            + inputType)
            +
            + inputDS
            + .countWindowAll(lowerbound,1)

              • End diff –

          From the [MS SQL Server documentation](https://msdn.microsoft.com/en-us/library/ms189461.aspx):
          > For example, ROWS BETWEEN 2 PRECEDING AND CURRENT ROW means that the window of rows that the function operates on is three rows in size, starting with 2 rows preceding until and including the current row.

          So, since `lowerBound` is already `AggregateUtil.getLowerBoundary(...) + 1`, we should be good.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106519205 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,76 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates, + inputType) + + inputDS + .countWindowAll(lowerbound,1) End diff – From the [MS SQL Server documentation] ( https://msdn.microsoft.com/en-us/library/ms189461.aspx): > For example, ROWS BETWEEN 2 PRECEDING AND CURRENT ROW means that the window of rows that the function operates on is three rows in size, starting with 2 rows preceding until and including the current row. So, since `lowerBound` is already `AggregateUtil.getLowerBoundary(...) + 1`, we should be good.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106526547

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

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + private[flink] def CreateBoundedProcessingOverGlobalWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): AllWindowFunction[Row,Row,GlobalWindow] = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregationStateType: RowTypeInfo =
          — End diff –

          not used

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106526547 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + private [flink] def CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): AllWindowFunction [Row,Row,GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = — End diff – not used
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106516596

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -17,35 +17,62 @@
          */
          package org.apache.flink.table.runtime.aggregate

          -import java.util
          +import scala.collection.JavaConversions.asScalaBuffer
          — End diff –

          Can we keep the imports as they are?

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106516596 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -17,35 +17,62 @@ */ package org.apache.flink.table.runtime.aggregate -import java.util +import scala.collection.JavaConversions.asScalaBuffer — End diff – Can we keep the imports as they are?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106528174

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
          +
          +class BoundedProcessingOverAllWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          + private var reuse: Row = _
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + if (null == accumulators)

          { + accumulators = new Row(aggregates.length) + }

          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + // setting the accumulators for each aggregation
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          accumulators can be reset with `aggregate.resetAccumulator(acc)`. So we can initialize once and reuse them

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106528174 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction + +class BoundedProcessingOverAllWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulators: Row = _ + private var reuse: Row = _ + + output = new Row(forwardedFieldCount + aggregates.length) + if (null == accumulators) { + accumulators = new Row(aggregates.length) + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + // setting the accumulators for each aggregation + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – accumulators can be reset with `aggregate.resetAccumulator(acc)`. So we can initialize once and reuse them
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106519460

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -130,32 +142,77 @@ class DataStreamOverAggregate(
          val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]

          val result: DataStream[Row] =

          • // partitioned aggregation
          • if (partitionKeys.nonEmpty) {
          • val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
          • namedAggregates,
          • inputType)
            + // partitioned aggregation
            + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - }
          • // non-partitioned aggregation
          • else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - }

            + } // non-partitioned aggregation
            + else

            { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

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

            { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + }

            // global non-partitioned aggregation
            + else {
            + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction(
            + namedAggregates,
            + inputType)
            +
            + inputDS
            + .countWindowAll(lowerbound,1)
            + .apply(windowFunction)
            + .setParallelism(1).setMaxParallelism(1)

              • End diff –

          Parallelism 1 is already implied by `countWindowAll`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106519460 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -130,32 +142,77 @@ class DataStreamOverAggregate( val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] val result: DataStream [Row] = // partitioned aggregation if (partitionKeys.nonEmpty) { val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( namedAggregates, inputType) + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType) - inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream[Row]] - } // non-partitioned aggregation else { - val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( - namedAggregates, - inputType, - false) - - inputDS - .process(processFunction).setParallelism(1).setMaxParallelism(1) - .returns(rowTypeInfo) - .name(aggOpName) - .asInstanceOf[DataStream[Row]] - } + } // non-partitioned aggregation + else { + val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction( + namedAggregates, + inputType, + false) + + inputDS + .process(processFunction).setParallelism(1).setMaxParallelism(1) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } + result + } + + def createBoundedAndCurrentRowProcessingTimeOverWindow( + inputDS: DataStream [Row] ): DataStream [Row] = { + + val overWindow: Group = logicWindow.groups.get(0) + val partitionKeys: Array [Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair [AggregateCall, String] ] = generateNamedAggregates + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + // window size is lowerbound +1 to comply with over semantics + val lowerbound: Int = AggregateUtil.getLowerBoundary( + logicWindow.constants, + overWindow.lowerBound, + getInput()) + 1 + + val result: DataStream [Row] = + // partitioned aggregation + if (partitionKeys.nonEmpty) { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverWindowFunction( + namedAggregates, + inputType) + inputDS + .keyBy(partitionKeys: _*) + .countWindow(lowerbound,1) + .apply(windowFunction) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + } // global non-partitioned aggregation + else { + val windowFunction = AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates, + inputType) + + inputDS + .countWindowAll(lowerbound,1) + .apply(windowFunction) + .setParallelism(1).setMaxParallelism(1) End diff – Parallelism 1 is already implied by `countWindowAll`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106517060

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

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          — End diff –

          Please align the arguments as the other functions do.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106517060 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], — End diff – Please align the arguments as the other functions do.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106534604

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

          { TableEnvironment, TableException }

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

          { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +}

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

          Each ITCase class adds significantly to the build time of the project because it starts a mini cluster which takes some time. Therefore, we try to keep the number of ITCase classes low and aim to test as much as possible with faster unit tests.

          Please add unit test methods that test the translation of queries to `org.apache.flink.table.api.scala.stream.sql.WindowAggregateTest`.

          Please add the following integration tests to `org.apache.flink.table.api.scala.stream.sql.SqlITCase`

          • multiple aggregation functions with partitioning
          • multiple aggregation functions without partitioning

          These tests classes contain already a few test methods that you can use as a starting point.

          Thank you

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106534604 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala — @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.api.scala.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api. { TableEnvironment, TableException } +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils. { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +} +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable +import org.apache.flink.types.Row + +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase { — End diff – Each ITCase class adds significantly to the build time of the project because it starts a mini cluster which takes some time. Therefore, we try to keep the number of ITCase classes low and aim to test as much as possible with faster unit tests. Please add unit test methods that test the translation of queries to `org.apache.flink.table.api.scala.stream.sql.WindowAggregateTest`. Please add the following integration tests to `org.apache.flink.table.api.scala.stream.sql.SqlITCase` multiple aggregation functions with partitioning multiple aggregation functions without partitioning These tests classes contain already a few test methods that you can use as a starting point. Thank you
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106523715

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

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + private[flink] def CreateBoundedProcessingOverGlobalWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): AllWindowFunction[Row,Row,GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverAllWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

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

          Please remove `;`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106523715 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + private [flink] def CreateBoundedProcessingOverGlobalWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): AllWindowFunction [Row,Row,GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverAllWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + def getLowerBoundary( + constants: ImmutableList [RexLiteral] , + lowerBound: RexWindowBound, + input: RelNode):Int = { + val ref: RexInputRef = lowerBound.getOffset.asInstanceOf [RexInputRef] + val index:Int = ref.getIndex + val count: Int = input.getRowType.getFieldCount + val lowerBoundIndex = count - index; — End diff – Please remove `;`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106538072

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]
          — End diff –

          What is the check `overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]` for and why do you check if `overWindow.upperBound.isPreceding`?

          Should the condition rather be like this:
          ```
          else if (overWindow.lowerBound.isPreceding() && !overWindow.lowerBound.isUnbounded() && // bounded preceding
          overWindow.upperBound.isCurrentRow() && // until current row
          overWindow.isRows) // is rows window
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106538072 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,9 +113,14 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] — End diff – What is the check `overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] ` for and why do you check if `overWindow.upperBound.isPreceding`? Should the condition rather be like this: ``` else if (overWindow.lowerBound.isPreceding() && !overWindow.lowerBound.isUnbounded() && // bounded preceding overWindow.upperBound.isCurrentRow() && // until current row overWindow.isRows) // is rows window ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106532760

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

          { TableEnvironment, TableException }

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

          { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +}

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

          This class is an integration tests because it starts a Flink minicluster. Hence its name should end with `ITCase` and not `Test` which would indicate a lightweight unit test.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106532760 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala — @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.api.scala.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api. { TableEnvironment, TableException } +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils. { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +} +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable +import org.apache.flink.types.Row + +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase { — End diff – This class is an integration tests because it starts a Flink minicluster. Hence its name should end with `ITCase` and not `Test` which would indicate a lightweight unit test.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106517261

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

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] =

          { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + }

          +
          + private[flink] def CreateBoundedProcessingOverGlobalWindowFunction(
          — End diff –

          indent -2

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106517261 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = + createDataSetAggregateBufferDataType(Array(), aggregates, inputType) + + new BoundedProcessingOverWindowFunction[GlobalWindow]( + aggregates, + aggFields, + inputType.getFieldCount) + + } + + private [flink] def CreateBoundedProcessingOverGlobalWindowFunction( — End diff – indent -2
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106520755

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

          { aggregationStateType) }

          }
          +
          + private[flink] def CreateBoundedProcessingOverWindowFunction(
          + namedAggregates: Seq[CalcitePair[AggregateCall, String]],
          + inputType: RelDataType): WindowFunction[Row, Row, Tuple, GlobalWindow] = {
          +
          + val (aggFields, aggregates) =
          + transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),
          + inputType,
          + needRetraction = false)
          +
          + val aggregationStateType: RowTypeInfo =
          — End diff –

          this is not used

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106520755 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -89,6 +116,58 @@ object AggregateUtil { aggregationStateType) } } + + private [flink] def CreateBoundedProcessingOverWindowFunction( + namedAggregates: Seq[CalcitePair [AggregateCall, String] ], + inputType: RelDataType): WindowFunction [Row, Row, Tuple, GlobalWindow] = { + + val (aggFields, aggregates) = + transformToAggregateFunctions( + namedAggregates.map(_.getKey), + inputType, + needRetraction = false) + + val aggregationStateType: RowTypeInfo = — End diff – this is not used
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106513895

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,41 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.plan.

          { RelOptCluster, RelTraitSet }

          — End diff –

          This PR has several changes that reformat the code but do not change the logic.
          In general, we try to limit PRs to the scope of the issue and avoid reformatting.

          1. It makes the PR harder to review because every reformatting change needs to be checked (was something changed or not). This is especially tiresome when indention is changed.
          2. It makes it more difficult to track changes in the history.
          3. Other users do not necessarily agree with your (or your IDE's) formatting and will format it back. So it might get back and forth.

          There is nothing wrong with adding a space between arguments here and there, but please keep reformatting changes to a minimum.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106513895 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,41 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.plan. { RelOptCluster, RelTraitSet } — End diff – This PR has several changes that reformat the code but do not change the logic. In general, we try to limit PRs to the scope of the issue and avoid reformatting. 1. It makes the PR harder to review because every reformatting change needs to be checked (was something changed or not). This is especially tiresome when indention is changed. 2. It makes it more difficult to track changes in the history. 3. Other users do not necessarily agree with your (or your IDE's) formatting and will format it back. So it might get back and forth. There is nothing wrong with adding a space between arguments here and there, but please keep reformatting changes to a minimum.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          One more thing. The PR includes a merge commit which makes squashing the commits and merging the PR a lot more difficult (see also the [contribution guidelines](http://flink.apache.org/contribute-code.html#coding-guidelines)).

          You can avoid merge commits if you branch from the master and just put commits on top. If you want to include somebody else's work, you can use `git cherry-pick` to pull over a commit. Use `git rebase` to rebase your commits on the latest master.

          Best, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3547 One more thing. The PR includes a merge commit which makes squashing the commits and merging the PR a lot more difficult (see also the [contribution guidelines] ( http://flink.apache.org/contribute-code.html#coding-guidelines )). You can avoid merge commits if you branch from the master and just put commits on top. If you want to include somebody else's work, you can use `git cherry-pick` to pull over a commit. Use `git rebase` to rebase your commits on the latest master. Best, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106566112

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class BoundedProcessingOverWindowFunction[W <: Window](
          — End diff –

          Hi @huawei-flink @fhueske in bounded OVER ROW case, I still recommend using processFunction. But we can just agree to disagree at current time. Because in the continuous progress of flip11, we may change our view.
          Best,
          SunJIncheng

          Show
          githubbot ASF GitHub Bot added a comment - Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106566112 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class BoundedProcessingOverWindowFunction [W <: Window] ( — End diff – Hi @huawei-flink @fhueske in bounded OVER ROW case, I still recommend using processFunction. But we can just agree to disagree at current time. Because in the continuous progress of flip11, we may change our view. Best, SunJIncheng
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106615434

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
          +
          +class BoundedProcessingOverAllWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          + private var reuse: Row = _
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + if (null == accumulators)

          { + accumulators = new Row(aggregates.length) + }

          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + // setting the accumulators for each aggregation
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          What is the meaning of the acc?
          What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106615434 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction + +class BoundedProcessingOverAllWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulators: Row = _ + private var reuse: Row = _ + + output = new Row(forwardedFieldCount + aggregates.length) + if (null == accumulators) { + accumulators = new Row(aggregates.length) + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + // setting the accumulators for each aggregation + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – What is the meaning of the acc? What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106615587

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala —
          @@ -0,0 +1,97 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
          +
          +class BoundedProcessingOverAllWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + Preconditions.checkNotNull(aggregates)
          + Preconditions.checkNotNull(aggFields)
          + Preconditions.checkArgument(aggregates.length == aggFields.length)
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          + private var reuse: Row = _
          +
          + output = new Row(forwardedFieldCount + aggregates.length)
          + if (null == accumulators)

          { + accumulators = new Row(aggregates.length) + }

          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + // setting the accumulators for each aggregation
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          What is the meaning of the acc?
          What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method
          should it be aggregates.resetAccumulator(aggregates)?

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106615587 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverAllWindowFunction.scala — @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable +import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction + +class BoundedProcessingOverAllWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var output: Row = _ + private var accumulators: Row = _ + private var reuse: Row = _ + + output = new Row(forwardedFieldCount + aggregates.length) + if (null == accumulators) { + accumulators = new Row(aggregates.length) + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + // setting the accumulators for each aggregation + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – What is the meaning of the acc? What should be passed to the accumulator for reset? logically I would expect that this is a non parameterized method should it be aggregates .resetAccumulator(aggregates )?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106615708

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,41 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.plan.

          { RelOptCluster, RelTraitSet }

          — End diff –

          @fhueske sorry about that. Will be more careful in the next one.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106615708 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,41 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.plan. { RelOptCluster, RelTraitSet } — End diff – @fhueske sorry about that. Will be more careful in the next one.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106616178

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]
          — End diff –

          I guess that is also a way to do it. The check allows to distinguish between time bounded and row bounded. I have no particular affection for my solution, it just worked. I will apply and test yours as well.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106616178 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,9 +113,14 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] — End diff – I guess that is also a way to do it. The check allows to distinguish between time bounded and row bounded. I have no particular affection for my solution, it just worked. I will apply and test yours as well.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106627282

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW
          + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[BasicSqlType]
          — End diff –

          Yes, I realized that when looking at PR #3550 that `isInstanceOf[BasicSqlType]` and `.isInstanceOf[IntervalSqlType]` distinguishes ROW from RANGE windows. I think using `.isRows()` is more clear and might also be safer because it appears to be a more public API than the type of the offset.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106627282 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,9 +113,14 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } // lowerbound is a BasicType and upperbound is PRECEEDING or CURRENT ROW + else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [BasicSqlType] — End diff – Yes, I realized that when looking at PR #3550 that `isInstanceOf [BasicSqlType] ` and `.isInstanceOf [IntervalSqlType] ` distinguishes ROW from RANGE windows. I think using `.isRows()` is more clear and might also be safer because it appears to be a more public API than the type of the offset.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106627390

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -17,34 +17,41 @@
          */
          package org.apache.flink.table.plan.nodes.datastream

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import org.apache.calcite.plan.

          { RelOptCluster, RelTraitSet }

          — End diff –

          No worries. I know that IDEs tend to reformat code but it really makes reviews harder.
          Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106627390 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,34 +17,41 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import org.apache.calcite.plan. { RelOptCluster, RelTraitSet } — End diff – No worries. I know that IDEs tend to reformat code but it really makes reviews harder. Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106868683

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

          { TableEnvironment, TableException }

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

          { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +}

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

          @fhueske Should I add all the test unit under the SqlITCase class?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106868683 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/ProcTimeRowBoundedAggregationTest.scala — @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.api.scala.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api. { TableEnvironment, TableException } +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.stream.utils. { + StreamingWithStateTestBase, + StreamITCase, + StreamTestData +} +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable +import org.apache.flink.types.Row + +class ProcTimeRowBoundedAggregationTest extends StreamingWithStateTestBase { — End diff – @fhueske Should I add all the test unit under the SqlITCase class?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3547#discussion_r106917566

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -17,35 +17,62 @@
          */
          package org.apache.flink.table.runtime.aggregate

          -import java.util
          +import scala.collection.JavaConversions.asScalaBuffer
          — End diff –

          I need that import, without it does not build.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3547#discussion_r106917566 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -17,35 +17,62 @@ */ package org.apache.flink.table.runtime.aggregate -import java.util +import scala.collection.JavaConversions.asScalaBuffer — End diff – I need that import, without it does not build.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          sorry about the mess. I don't understand the mess the rebase does with eclipse... I will close this PR and open another one including all the changes and comments

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3547 sorry about the mess. I don't understand the mess the rebase does with eclipse... I will close this PR and open another one including all the changes and comments
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink closed the pull request at:

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

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

          Github user huawei-flink commented on the issue:

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

          @fhueske let's see if the third attempt works. I have included all the comments of @sunjincheng121 (apart from the window function) and having done a merge on a fresh checkout also the imports should be fine.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3574 @fhueske let's see if the third attempt works. I have included all the comments of @sunjincheng121 (apart from the window function) and having done a merge on a fresh checkout also the imports should be fine.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testBoundedNonPartitionedProcessingWindowWithRow() =

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

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

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

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

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

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

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

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

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

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

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

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

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

          { (aggFunction, accumulatorRowType, aggResultRowType) }

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

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

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

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

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

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

          {ArrayList => JArrayList, List => JList}

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

          {AggregateFunction => DataStreamAggOverFunc}

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

          {Accumulator, AggregateFunction}

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

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106965395 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.aggregate + +import java.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ + + override def createAccumulator(): Row = { + val accumulatorRow: Row = new Row(aggregates.length) — End diff – The accumulator row must include the forwarded fields to checkpoint the last received row.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

          {ArrayList => JArrayList, List => JList}

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

          {AggregateFunction => DataStreamAggOverFunc}

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

          {Accumulator, AggregateFunction}

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

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106965219 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.aggregate + +import java.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ — End diff – We cannot save the last record in a member variable. This will be lost in case of a failure. We have to put it in the accumulator such that it is checkpointed.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

          {ArrayList => JArrayList, List => JList}

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

          {AggregateFunction => DataStreamAggOverFunc}

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

          {Accumulator, AggregateFunction}

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

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

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

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

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

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

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

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

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

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106966705 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.aggregate + +import java.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ + + override def createAccumulator(): Row = { + val accumulatorRow: Row = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulatorRow.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + accumulatorRow + } + + override def add(value: Row, accumulatorRow: Row): Unit = { + var i = 0 + while (i < aggregates.length) { + val acc = accumulatorRow.getField(i).asInstanceOf[Accumulator] + val v = value.getField(aggFields(i)) + aggregates(i).accumulate(acc, v) + i += 1 + } + lastVal = value; + } + + override def getResult(accumulatorRow: Row): Row = { + val output = new Row(forwardedFieldCount + aggFields.length) + + var i = 0 + // set the output value of forward fields + while (i < forwardedFieldCount) { + output.setField(i, lastVal.getField(i)) + i += 1 + } + + i = 0 + while (i < aggregates.length) { + val acc = accumulatorRow.getField(i).asInstanceOf[Accumulator] + output.setField(forwardedFieldCount + i, aggregates(i).getValue(acc)) + i += 1 + } + output + } + + override def merge(aAccumulatorRow: Row, bAccumulatorRow: Row): Row = { + + var i = 0 + while (i < aggregates.length) { — End diff – I'm sorry, I just realized that we cannot use the `AggregateFunction` interface. We would need to merge the forwarded fields as well. Since we do not know which accumulator holds the forwarded fields of the last row, it is not possible to use this interface. I'm sorry that I led you into this direction.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

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

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

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

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

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

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

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

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

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

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

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

          { out.collect(iter.next) }
          }

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

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

          }
          }
          ```

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106962344 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala — @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class IncrementalAggregateOverWindowFunction [W <: Window] ( + private val numGroupingKey: Int, + private val numAggregates: Int, + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + + private var output: Row = _ + private var reuse: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + numAggregates) + } + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next — End diff – This function copies each field of the intermediate row into the output row. I think we can merge `IncrementalAggregateOverWindowFunction` and `IncrementalAggregateOverAllWindowFunction` into a class called `ForwardWindowFunction` which is defined as below: ``` class ForwardWindowFunction [W <: Window] () extends WindowFunction [Row, Row, Tuple, W] with AllWindowFunction [Row, Row, W] { override def apply( key: Tuple, window: W, records: Iterable [Row] , out: Collector [Row] ): Unit = { val iter = records.iterator while (iter.hasNext) { out.collect(iter.next) } } override def apply(window: W, records: Iterable [Row] , out: Collector [Row] ): Unit = { val iter = records.iterator while (iter.hasNext) { out.collect(iter.next) } } } ```
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

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

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

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

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

          This should give us good coverage of the feature.

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

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

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

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

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

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

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

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

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

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

          {ArrayList => JArrayList, List => JList}

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

          {AggregateFunction => DataStreamAggOverFunc}

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

          {Accumulator, AggregateFunction}

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

          +space `lastVal: Row`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r106964910 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggOverFunction.scala — @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.aggregate + +import java.util. {ArrayList => JArrayList, List => JList} +import org.apache.flink.api.common.functions. {AggregateFunction => DataStreamAggOverFunc} +import org.apache.flink.table.functions. {Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.Preconditions + +/** + * Aggregate Function used for the aggregate operator in + * [ [org.apache.flink.streaming.api.datastream.WindowedStream] ] + * + * @param aggregates the list of all [ [org.apache.flink.table.functions.AggregateFunction] ] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount the number of elements in the Row to kept after the aggregation + */ +class AggregateAggOverFunction( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends DataStreamAggOverFunc [Row, Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var lastVal:Row = _ — End diff – +space `lastVal: Row`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

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

          `aggString` is not used.

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

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

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

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

          { ) streamUtil.verifySql(sql, expected) }

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

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

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

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

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

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

          { aggregationStateType) }

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

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

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

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

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

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

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

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

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

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

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

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

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

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

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

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

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

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

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

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

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

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

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

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

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on a diff in the pull request: https://github.com/apache/flink/pull/3574#discussion_r107103813 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala — @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import org.apache.flink.api.java.tuple.Tuple +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.util.Collector +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.util.Preconditions +import org.apache.flink.table.functions.Accumulator +import java.lang.Iterable + +class IncrementalAggregateOverWindowFunction [W <: Window] ( + private val numGroupingKey: Int, + private val numAggregates: Int, + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + + private var output: Row = _ + private var reuse: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + numAggregates) + } + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next — End diff – I agree with you, the logic is exactly the same. Wanted to be sure the PR was "correct" first. I am not used to the fact that SCALA allows for multiple class extension. :-P
          Hide
          githubbot ASF GitHub Bot added a comment -

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

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

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

          { ) streamUtil.verifySql(sql, expected) }

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

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

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

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

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

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

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

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

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

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

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

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

          { aggregationStateType) }

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

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

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

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

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

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

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

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

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

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

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

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

          { aggregationStateType) }

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

          +space

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

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

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

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

          { aggregationStateType) }

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

          rm `;`

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

          Github user huawei-flink commented on the issue:

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

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

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

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

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

          Github user huawei-flink commented on the issue:

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

          @fhueske @sunjincheng121 would a ProcessWindowFunction work?

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

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

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

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

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

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

          Yes, your right.

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

          Github user huawei-flink commented on the issue:

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

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

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

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

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

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

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

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

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

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

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

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateOverWindowFunction.scala —
          @@ -0,0 +1,73 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import org.apache.flink.api.java.tuple.Tuple
          +import org.apache.flink.types.Row
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.util.Collector
          +import org.apache.flink.table.functions.AggregateFunction
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.util.Preconditions
          +import org.apache.flink.table.functions.Accumulator
          +import java.lang.Iterable
          +
          +class IncrementalAggregateOverWindowFunction[W <: Window](
          + private val numGroupingKey: Int,
          + private val numAggregates: Int,
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          + private var output: Row = _
          + private var reuse: Row = _
          +
          + override def open(parameters: Configuration): Unit =

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

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

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