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

Add processing time OVER RANGE 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 RANGE 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() RANGE BETWEEN INTERVAL '1' HOUR PRECEDING AND CURRENT ROW) AS sumB,
        MIN(b) OVER (PARTITION BY c ORDER BY procTime() RANGE BETWEEN INTERVAL '1' HOUR 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-5657)
      • 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
          rtudoran radu added a comment -

          I would like to implement the this issue. I already have a working prototype internally for it which can speed up the process.

          Show
          rtudoran radu added a comment - I would like to implement the this issue. I already have a working prototype internally for it which can speed up the process.
          Hide
          rtudoran radu added a comment -

          Discussion about the comment Fabian made in the mail tread for the issues:
          "
          @Radu: So far we had very coarse grained DataStreamRelNodes (e.g., DataStreamAggregate implements tumbling, sliding, and session windows for processing and event time). However, it might make sense to start implementing more fine-grained DataStreamRelNodes.
          "

          I would assume you refer to this as having a hierarchy in the translation rule, so instead of mapping

          DataStreamAggregateRule.scala
          (mapping all to)->
          DataStreamAggreagte

          we should split based on the specific scenario and type of processing. For example for this case the rule should translate to:
          DataStreamSlidingProcessingTime
          ..which would contain the actual implementation

          Show
          rtudoran radu added a comment - Discussion about the comment Fabian made in the mail tread for the issues: " @Radu: So far we had very coarse grained DataStreamRelNodes (e.g., DataStreamAggregate implements tumbling, sliding, and session windows for processing and event time). However, it might make sense to start implementing more fine-grained DataStreamRelNodes. " I would assume you refer to this as having a hierarchy in the translation rule, so instead of mapping DataStreamAggregateRule.scala (mapping all to)-> DataStreamAggreagte we should split based on the specific scenario and type of processing. For example for this case the rule should translate to: DataStreamSlidingProcessingTime ..which would contain the actual implementation
          Hide
          fhueske Fabian Hueske added a comment -

          Yes, that's what I meant.

          Show
          fhueske Fabian Hueske added a comment - Yes, that's what I meant.
          Hide
          rtudoran radu added a comment -

          I would like to bring into the discussion two items:

          1) For the query semantic I see that in the example the interval is strongly defined
          "RANGE BETWEEN INTERVAL '1' HOUR PRECEDING AND CURRENT ROW"
          I would propose to support also the syntax (used commonly in Calcite)
          "RANGE INTERVAL '1' HOUR PRECEDING"
          This syntax implicitly expects that the reference is the current row.

          2)◾The PARTITION BY clause is optional (no partitioning results in single threaded execution).
          Is there a strong preference to use for the single tread execution one of the two options:
          -globalwindow (windowAll) - I would expect this to be used
          -keyBy(constantFunction).window().setParalelism(1) - this idea would keep things symmetric and with a unique implementation for the two cases (with partition and without). The disadvantage is that we create an addition operator with a dummy functionality (i.e., a keyBy with a constant function to ensure that all events arrive in the same place) to maintain the consistency in the implementation.

          Show
          rtudoran radu added a comment - I would like to bring into the discussion two items: 1) For the query semantic I see that in the example the interval is strongly defined "RANGE BETWEEN INTERVAL '1' HOUR PRECEDING AND CURRENT ROW" I would propose to support also the syntax (used commonly in Calcite) "RANGE INTERVAL '1' HOUR PRECEDING" This syntax implicitly expects that the reference is the current row. 2)◾The PARTITION BY clause is optional (no partitioning results in single threaded execution). Is there a strong preference to use for the single tread execution one of the two options: -globalwindow (windowAll) - I would expect this to be used -keyBy(constantFunction).window().setParalelism(1) - this idea would keep things symmetric and with a unique implementation for the two cases (with partition and without). The disadvantage is that we create an addition operator with a dummy functionality (i.e., a keyBy with a constant function to ensure that all events arrive in the same place) to maintain the consistency in the implementation.
          Hide
          fhueske Fabian Hueske added a comment -

          Ad 1) The restrictions was only meant for the supported semantics. If RANGE INTERVAL '1' HOUR PRECEDING is equivalent to RANGE BETWEEN INTERVAL '1' HOUR PRECEDING AND CURRENT ROW it will be automatically supported because we do the check on the logical operator level and not on the syntactic level, i.e., we won't notice which of both variants was used.

          Ad 2) I'd prefer the first option, i.e., using windowAll() this will make the behavior more transparent for the lower levels of Flink.

          Show
          fhueske Fabian Hueske added a comment - Ad 1) The restrictions was only meant for the supported semantics. If RANGE INTERVAL '1' HOUR PRECEDING is equivalent to RANGE BETWEEN INTERVAL '1' HOUR PRECEDING AND CURRENT ROW it will be automatically supported because we do the check on the logical operator level and not on the syntactic level, i.e., we won't notice which of both variants was used. Ad 2) I'd prefer the first option, i.e., using windowAll() this will make the behavior more transparent for the lower levels of Flink.
          Hide
          rtudoran radu added a comment -

          There is another decision point to be made. When we do the parsing of the SQL query two options are possible:

          1) the entire expression is translated to a LogicalCalc
          An example is
          rel#8:LogicalCalc.NONE(input=rel#2:Subset#0.NONE,expr#0..5=

          {inputs}

          ,expr#6=17:34:58,expr#7=3600000,expr#8=COUNT($t5) OVER (PARTITION BY $t2 ORDER BY $t6 RANGE BETWEEN $t7 PRECEDING AND CURRENT ROW),expr#9=0,expr#10=>($t8, $t9),expr#11=$SUM0($t5) OVER (PARTITION BY $t2 ORDER BY $t6 RANGE BETWEEN $t7 PRECEDING AND CURRENT ROW),expr#12=CAST($t11):DOUBLE,expr#13=null,expr#14=CASE($t10, $t12, $t13),timeevent=$t0,sumB=$t14)

          Notice that the logic related to window (boundaries, aggregates,partition over clause...) are Rex objects in the LogicalCalc (e.g. RexOver, RexWindow...)

          2) the other option is that the whole logic of the over clause to be mapped directly to a LogicalWindow operator:
          rel#14:LogicalWindow.NONE(input=rel#13:Subset#2.NONE,window#0=window(partition

          {1}

          order by [] range between $4 PRECEDING and CURRENT ROW aggs [COUNT($2), $SUM0($2)]))

          Notice that all the information specific to the window processing is now in available in the LogicalWindow object, while the other information related to projection or calc will have a dedicated LogicalCalc node/operator.

          From my point of view it is more elegant and the code is more robust with the second option. This is what i propose to be used (and the option that i selected).
          However, based on default rules adopted from Calcite in Flink, the 2 rules that enable to translate a query to the second option are not enabled.
          That is why i added the 2 rules to the FlinkRuleSet

          ProjectWindowTransposeRule.INSTANCE,
          ProjectToWindowRule.INSTANCE,

          I will push these modifications together with the rest of the code. However, if anyone has a strong preference for one option versus the other, or there is a specific reason that i am missing for which the two rules were not enabled in the first place, please let me know

          Show
          rtudoran radu added a comment - There is another decision point to be made. When we do the parsing of the SQL query two options are possible: 1) the entire expression is translated to a LogicalCalc An example is rel#8:LogicalCalc.NONE(input=rel#2:Subset#0.NONE,expr#0..5= {inputs} ,expr#6=17:34:58,expr#7=3600000,expr#8=COUNT($t5) OVER (PARTITION BY $t2 ORDER BY $t6 RANGE BETWEEN $t7 PRECEDING AND CURRENT ROW),expr#9=0,expr#10=>($t8, $t9),expr#11=$SUM0($t5) OVER (PARTITION BY $t2 ORDER BY $t6 RANGE BETWEEN $t7 PRECEDING AND CURRENT ROW),expr#12=CAST($t11):DOUBLE,expr#13=null,expr#14=CASE($t10, $t12, $t13),timeevent=$t0,sumB=$t14) Notice that the logic related to window (boundaries, aggregates,partition over clause...) are Rex objects in the LogicalCalc (e.g. RexOver, RexWindow...) 2) the other option is that the whole logic of the over clause to be mapped directly to a LogicalWindow operator: rel#14:LogicalWindow.NONE(input=rel#13:Subset#2.NONE,window#0=window(partition {1} order by [] range between $4 PRECEDING and CURRENT ROW aggs [COUNT($2), $SUM0($2)] )) Notice that all the information specific to the window processing is now in available in the LogicalWindow object, while the other information related to projection or calc will have a dedicated LogicalCalc node/operator. From my point of view it is more elegant and the code is more robust with the second option. This is what i propose to be used (and the option that i selected). However, based on default rules adopted from Calcite in Flink, the 2 rules that enable to translate a query to the second option are not enabled. That is why i added the 2 rules to the FlinkRuleSet ProjectWindowTransposeRule.INSTANCE, ProjectToWindowRule.INSTANCE, I will push these modifications together with the rest of the code. However, if anyone has a strong preference for one option versus the other, or there is a specific reason that i am missing for which the two rules were not enabled in the first place, please let me know
          Hide
          fhueske Fabian Hueske added a comment -

          I agree radu, the second approach looks much easier to handle at optimization time.
          It is (usually) not a problem to add rules as needed.

          Thanks, Fabian

          Show
          fhueske Fabian Hueske added a comment - I agree radu , the second approach looks much easier to handle at optimization time. It is (usually) not a problem to add rules as needed. Thanks, Fabian
          Hide
          rtudoran radu added a comment -

          I have another dilemma for the implementation. When we build the window functions to compute the aggregates we have 2 options:

          1) we extract the input data stream and then apply directly the window logic on top just like we would work with the datastream api. For this scenario all parameters of the sql operator will be passed in the constructor of the window apply function
          e.g.
          inputDS=..
          inputDS.windowAssigner().trigger().evictor().
          apply(new WindowFunction(maxField)

          { //max aggregate max = ... Raw.getField(maxField) //sum ... //min ... }

          )

          2) the other option is to follow the design from Calc operator where we generate a text description of the window function for which we construct a windowRunner and use janino to compile it. This can slightly optimize things as some aggregation logic that might not be useful will not be implemented in the function (rather then in the first option when we would carry everything all the time

          Do you have any preferences for the implementation?
          Option 1 is more clean for maintenance while option 2 is more efficient

          Show
          rtudoran radu added a comment - I have another dilemma for the implementation. When we build the window functions to compute the aggregates we have 2 options: 1) we extract the input data stream and then apply directly the window logic on top just like we would work with the datastream api. For this scenario all parameters of the sql operator will be passed in the constructor of the window apply function e.g. inputDS=.. inputDS.windowAssigner().trigger().evictor(). apply(new WindowFunction(maxField) { //max aggregate max = ... Raw.getField(maxField) //sum ... //min ... } ) 2) the other option is to follow the design from Calc operator where we generate a text description of the window function for which we construct a windowRunner and use janino to compile it. This can slightly optimize things as some aggregation logic that might not be useful will not be implemented in the function (rather then in the first option when we would carry everything all the time Do you have any preferences for the implementation? Option 1 is more clean for maintenance while option 2 is more efficient
          Hide
          wheat9 Haohui Mai added a comment -

          I may not fully get the question – Is it possible to do something similar to LogicalWindowAggregate? Essentially the operator will look like this:

          LogicalWindowAggregate(window=[..], agg={MAX($0)})
            LogicalProject(...)
          

          I assume that if you implement the translateToPlan function similar to the DataStreamAggregate then you don't need to take care of the code generation part?

          Show
          wheat9 Haohui Mai added a comment - I may not fully get the question – Is it possible to do something similar to LogicalWindowAggregate ? Essentially the operator will look like this: LogicalWindowAggregate(window=[..], agg={MAX($0)}) LogicalProject(...) I assume that if you implement the translateToPlan function similar to the DataStreamAggregate then you don't need to take care of the code generation part?
          Hide
          rtudoran radu added a comment -

          Thanks Haohui for the suggestion. You are right with the suggestions you make and this is the path i took. However, I was referring to what should happen in the translateToPlan function. There you have the 2 options i mentioned to build the logic to compute the aggregate

          Show
          rtudoran radu added a comment - Thanks Haohui for the suggestion. You are right with the suggestions you make and this is the path i took. However, I was referring to what should happen in the translateToPlan function. There you have the 2 options i mentioned to build the logic to compute the aggregate
          Hide
          fhueske Fabian Hueske added a comment -

          Eventually, we will go for code generation. However, we are also in the process of changing the interface of the aggregation functions (see FLINK-5564). So, for now it should be good enough to implement a generic, configurable WindowFunction.

          Show
          fhueske Fabian Hueske added a comment - Eventually, we will go for code generation. However, we are also in the process of changing the interface of the aggregation functions (see FLINK-5564 ). So, for now it should be good enough to implement a generic, configurable WindowFunction.
          Hide
          rtudoran radu added a comment -

          When we implement the aggregation functions we could use the interface that flink already provides (Aggregator<T1,T2>). However, if we go this way then we need to provide many implementations for all aggregation functions (6) and for all types of SQL.
          Is this the right way to go for this?

          Show
          rtudoran radu added a comment - When we implement the aggregation functions we could use the interface that flink already provides (Aggregator<T1,T2>). However, if we go this way then we need to provide many implementations for all aggregation functions (6) and for all types of SQL. Is this the right way to go for this?
          Hide
          sunjincheng121 sunjincheng added a comment - - edited

          HI,@radu At present the new aggregation interface has been completed. You can see the details
          And see more info from the link above,which Fabian Hueske give you.

          Show
          sunjincheng121 sunjincheng added a comment - - edited HI,@radu At present the new aggregation interface has been completed. You can see the details And see more info from the link above,which Fabian Hueske give you.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user huawei-flink opened a pull request:

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

          FLINK-5654 Add processing time OVER RANGE 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-5654

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

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


          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 4e8ad831a04347bb94fae23cbc576f23e81fca9d
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-10T16:21:56Z

          small progress for global window

          commit 6079d9ede35ffa6297130a08764fcb69d2609fbd
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-10T16:28:31Z

          Merg conflict solved

          commit 9ec8edf17988b5d2ac2e5fa428854ef9c99be1e4
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-10T16:37:04Z

          Build errors solved

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

          fixing unused imports

          commit da4992a7dccc1f7022c9adaca9adf70c56d36f8f
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-28T09:42:08Z

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

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

          Adding aggregation functions in new package

          commit 6b5aa7cc40e72a3bbd67b235fc6c2f75f2e6804b
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-02-28T15:53:14Z

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

          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


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user huawei-flink opened a pull request: https://github.com/apache/flink/pull/3459 FLINK-5654 Add processing time OVER RANGE 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-5654 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3459.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 #3459 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 4e8ad831a04347bb94fae23cbc576f23e81fca9d Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-10T16:21:56Z small progress for global window commit 6079d9ede35ffa6297130a08764fcb69d2609fbd Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-10T16:28:31Z Merg conflict solved commit 9ec8edf17988b5d2ac2e5fa428854ef9c99be1e4 Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-10T16:37:04Z Build errors solved commit 2d6ea54cdfe7096fcd6bbb8ae8bbdf2cd2ba2d0c Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-10T16:53:03Z fixing unused imports commit da4992a7dccc1f7022c9adaca9adf70c56d36f8f Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-28T09:42:08Z Merge branch ' FLINK-5653 ' of https://github.com/huawei-flink/flink into FLINK-5654 commit 40f5ff80bb3023ce1125094dca135bc230ab67ba Author: Stefano Bortoli <s.bortoli@gmail.com> Date: 2017-02-28T15:52:35Z Adding aggregation functions in new package commit 6b5aa7cc40e72a3bbd67b235fc6c2f75f2e6804b Author: rtudoran <tudoranradu@ymail.com> Date: 2017-02-28T15:53:14Z Merge branch ' FLINK-5653 ' of https://github.com/huawei-flink/flink into FLINK-5654 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
          Hide
          rtudoran radu added a comment -

          Thanks sunjincheng - I saw the message to late unfortunately. I can do another pull request after the commit is merge to update the aggregations to your interface.

          Show
          rtudoran radu added a comment - Thanks sunjincheng - I saw the message to late unfortunately. I can do another pull request after the commit is merge to update the aggregations to your interface.
          Hide
          sunjincheng121 sunjincheng added a comment - - edited

          Hi, @radu you are welcome. Currently you can rebase the code on the master.
          Thanks,
          SunJincheng

          Show
          sunjincheng121 sunjincheng added a comment - - edited Hi, @radu you are welcome. Currently you can rebase the code on the master. Thanks, SunJincheng
          Hide
          rtudoran radu added a comment -

          Hi,

          I did the rebase yesterday before I did the pull request. However, the aggregation function are an different implementation. That is why I am saying that if we want to clean this up we can go for a second jira - which would not be a problem.

          Show
          rtudoran radu added a comment - Hi, I did the rebase yesterday before I did the pull request. However, the aggregation function are an different implementation. That is why I am saying that if we want to clean this up we can go for a second jira - which would not be a problem.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3459#discussion_r104282393

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          +
          + private LogicalWindow windowReference;
          + private String description;
          — End diff –

          I didn't find where `description` is used. If one variable is not used, it shouldn't be declared.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3459#discussion_r104282393 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { + + private LogicalWindow windowReference; + private String description; — End diff – I didn't find where `description` is used. If one variable is not used, it shouldn't be declared.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user shijinkui commented on the issue:

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

          @stefanobortoli One PR should has only one commit as soon as possible. If having so many commit, it's hard to review..

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on the issue: https://github.com/apache/flink/pull/3459 @stefanobortoli One PR should has only one commit as soon as possible. If having so many commit, it's hard to review..
          Hide
          rtudoran radu added a comment -

          @shijinkui - thanks for the comment
          I keeped description because all other sql operator implementation have it. It was more for uniformity. Additionally, if at some point we decide to go for compiling functions instead of working directly against the datastream API, than this is typically used for naming. Because of these I propose to keep it.

          Show
          rtudoran radu added a comment - @shijinkui - thanks for the comment I keeped description because all other sql operator implementation have it. It was more for uniformity. Additionally, if at some point we decide to go for compiling functions instead of working directly against the datastream API, than this is typically used for naming. Because of these I propose to keep 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/3459#discussion_r104362378

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          +
          + private LogicalWindow windowReference;
          + private String description;
          — End diff –

          @rtudoran you should have a look at this.

          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/3459#discussion_r104362378 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { + + private LogicalWindow windowReference; + private String description; — End diff – @rtudoran you should have a look at this.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @shijinkui I think Radu and I complicated a bit the code pulling from each other branch some stuff. Next time we'll squash the commit.

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3459 @shijinkui I think Radu and I complicated a bit the code pulling from each other branch some stuff. Next time we'll squash the commit.
          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/3459#discussion_r104366556

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          +
          + private LogicalWindow windowReference;
          + private String description;
          — End diff –

          @shijinkui - thanks for the comment
          I kept description because all other sql operator implementation have it. It was more for uniformity. Additionally, if at some point we decide to go for compiling functions instead of working directly against the datastream API, than this is typically used for naming. Because of these I propose to keep 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/3459#discussion_r104366556 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { + + private LogicalWindow windowReference; + private String description; — End diff – @shijinkui - thanks for the comment I kept description because all other sql operator implementation have it. It was more for uniformity. Additionally, if at some point we decide to go for compiling functions instead of working directly against the datastream API, than this is typically used for naming. Because of these I propose to keep it.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          Related to the building failure - I see that this fails only for one particular case. I looked into the error and it is not related to my modifications as you cans see below. In fact I did not touch on the Cassandra connector which is the one failing nor I caused, I would say, any things to conflict with it.
          From my point of view this could be pulled in

          [INFO] flink-libraries .................................... SUCCESS [ 0.271 s]
          [INFO] flink-table ........................................ SUCCESS [02:44 min]
          [INFO] flink-jdbc ......................................... SUCCESS [ 0.898 s]
          [INFO] flink-hbase ........................................ SUCCESS [ 48.336 s]
          [INFO] flink-hcatalog ..................................... SUCCESS [ 8.864 s]
          [INFO] flink-metrics-jmx .................................. SUCCESS [ 0.487 s]
          [INFO] flink-connector-kafka-base ......................... SUCCESS [ 4.050 s]
          [INFO] flink-connector-kafka-0.8 .......................... SUCCESS [ 3.325 s]
          [INFO] flink-connector-kafka-0.9 .......................... SUCCESS [ 3.302 s]
          [INFO] flink-connector-kafka-0.10 ......................... SUCCESS [ 1.495 s]
          [INFO] flink-connector-elasticsearch-base ................. SUCCESS [ 5.535 s]
          [INFO] flink-connector-elasticsearch ...................... SUCCESS [01:07 min]
          [INFO] flink-connector-elasticsearch2 ..................... SUCCESS [ 14.613 s]
          [INFO] flink-connector-rabbitmq ........................... SUCCESS [ 0.493 s]
          [INFO] flink-connector-twitter ............................ SUCCESS [ 2.241 s]
          [INFO] flink-connector-nifi ............................... SUCCESS [ 0.816 s]
          [INFO] flink-connector-cassandra .......................... FAILURE [02:15 min]
          [INFO] flink-connector-filesystem ......................... SKIPPED
          [INFO] flink-connector-kinesis ............................ SKIPPED
          [INFO] flink-connector-elasticsearch5 ..................... SKIPPED
          [INFO] flink-examples-streaming ........................... SKIPPED
          [INFO] flink-gelly ........................................ SKIPPED
          [INFO] flink-gelly-scala .................................. SKIPPED
          [INFO] flink-gelly-examples ............................... SKIPPED
          [INFO] flink-python ....................................... SKIPPED
          [INFO] flink-ml ........................................... SKIPPED
          [INFO] flink-cep .......................................... SKIPPED
          [INFO] flink-cep-scala .................................... SKIPPED
          [INFO] flink-scala-shell .................................. SKIPPED
          [INFO] flink-quickstart ................................... SKIPPED
          [INFO] flink-quickstart-java .............................. SKIPPED
          [INFO] flink-quickstart-scala ............................. SKIPPED
          [INFO] flink-storm ........................................ SKIPPED
          [INFO] flink-storm-examples ............................... SKIPPED
          [INFO] flink-streaming-contrib ............................ SKIPPED
          [INFO] flink-tweet-inputformat ............................ SKIPPED
          [INFO] flink-connector-wikiedits .......................... SKIPPED
          [INFO] flink-mesos ........................................ SKIPPED
          [INFO] flink-yarn ......................................... SKIPPED
          [INFO] flink-metrics-dropwizard ........................... SKIPPED
          [INFO] flink-metrics-ganglia .............................. SKIPPED
          [INFO] flink-metrics-graphite ............................. SKIPPED
          [INFO] flink-metrics-statsd ............................... SKIPPED
          [INFO] flink-dist ......................................... SKIPPED
          [INFO] flink-fs-tests ..................................... SKIPPED
          [INFO] flink-yarn-tests ................................... SKIPPED
          [INFO] ------------------------------------------------------------------------
          [INFO] BUILD FAILURE
          [INFO] ------------------------------------------------------------------------
          [INFO] Total time: 25:31 min
          [INFO] Finished at: 2017-03-06T12:07:47+00:00
          [INFO] Final Memory: 161M/493M
          [INFO] ------------------------------------------------------------------------
          [ERROR] Failed to execute goal org.apache.maven.plugins:maven-surefire-plugin:2.18.1:test (integration-tests) on project flink-connector-cassandra_2.10: There are test failures.
          [ERROR]
          [ERROR] Please refer to /home/travis/build/apache/flink/flink-connectors/flink-connector-cassandra/target/surefire-reports for the individual test results.
          [ERROR] -> [Help 1]
          [ERROR]
          [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.
          [ERROR] Re-run Maven using the -X switch to enable full debug logging.
          [ERROR]
          [ERROR] For more information about the errors and possible solutions, please read the following articles:
          [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException
          [ERROR]
          [ERROR] After correcting the problems, you can resume the build with the command
          [ERROR] mvn <goals> -rf :flink-connector-cassandra_2.10
          Trying to KILL watchdog (1345).
          ./tools/travis_mvn_watchdog.sh: line 210: 1345 Terminated watchdog
          MVN exited with EXIT CODE: 1.
          java.io.FileNotFoundException: build-target/lib/flink-dist-*.jar (No such file or directory)
          at java.util.zip.ZipFile.open(Native Method)
          at java.util.zip.ZipFile.<init>(ZipFile.java:220)
          at java.util.zip.ZipFile.<init>(ZipFile.java:150)
          at java.util.zip.ZipFile.<init>(ZipFile.java:121)
          at sun.tools.jar.Main.list(Main.java:1060)
          at sun.tools.jar.Main.run(Main.java:291)
          at sun.tools.jar.Main.main(Main.java:1233)
          find: `./flink-yarn-tests/target/flink-yarn-tests*': No such file or directory

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3459 Related to the building failure - I see that this fails only for one particular case. I looked into the error and it is not related to my modifications as you cans see below. In fact I did not touch on the Cassandra connector which is the one failing nor I caused, I would say, any things to conflict with it. From my point of view this could be pulled in [INFO] flink-libraries .................................... SUCCESS [ 0.271 s] [INFO] flink-table ........................................ SUCCESS [02:44 min] [INFO] flink-jdbc ......................................... SUCCESS [ 0.898 s] [INFO] flink-hbase ........................................ SUCCESS [ 48.336 s] [INFO] flink-hcatalog ..................................... SUCCESS [ 8.864 s] [INFO] flink-metrics-jmx .................................. SUCCESS [ 0.487 s] [INFO] flink-connector-kafka-base ......................... SUCCESS [ 4.050 s] [INFO] flink-connector-kafka-0.8 .......................... SUCCESS [ 3.325 s] [INFO] flink-connector-kafka-0.9 .......................... SUCCESS [ 3.302 s] [INFO] flink-connector-kafka-0.10 ......................... SUCCESS [ 1.495 s] [INFO] flink-connector-elasticsearch-base ................. SUCCESS [ 5.535 s] [INFO] flink-connector-elasticsearch ...................... SUCCESS [01:07 min] [INFO] flink-connector-elasticsearch2 ..................... SUCCESS [ 14.613 s] [INFO] flink-connector-rabbitmq ........................... SUCCESS [ 0.493 s] [INFO] flink-connector-twitter ............................ SUCCESS [ 2.241 s] [INFO] flink-connector-nifi ............................... SUCCESS [ 0.816 s] [INFO] flink-connector-cassandra .......................... FAILURE [02:15 min] [INFO] flink-connector-filesystem ......................... SKIPPED [INFO] flink-connector-kinesis ............................ SKIPPED [INFO] flink-connector-elasticsearch5 ..................... SKIPPED [INFO] flink-examples-streaming ........................... SKIPPED [INFO] flink-gelly ........................................ SKIPPED [INFO] flink-gelly-scala .................................. SKIPPED [INFO] flink-gelly-examples ............................... SKIPPED [INFO] flink-python ....................................... SKIPPED [INFO] flink-ml ........................................... SKIPPED [INFO] flink-cep .......................................... SKIPPED [INFO] flink-cep-scala .................................... SKIPPED [INFO] flink-scala-shell .................................. SKIPPED [INFO] flink-quickstart ................................... SKIPPED [INFO] flink-quickstart-java .............................. SKIPPED [INFO] flink-quickstart-scala ............................. SKIPPED [INFO] flink-storm ........................................ SKIPPED [INFO] flink-storm-examples ............................... SKIPPED [INFO] flink-streaming-contrib ............................ SKIPPED [INFO] flink-tweet-inputformat ............................ SKIPPED [INFO] flink-connector-wikiedits .......................... SKIPPED [INFO] flink-mesos ........................................ SKIPPED [INFO] flink-yarn ......................................... SKIPPED [INFO] flink-metrics-dropwizard ........................... SKIPPED [INFO] flink-metrics-ganglia .............................. SKIPPED [INFO] flink-metrics-graphite ............................. SKIPPED [INFO] flink-metrics-statsd ............................... SKIPPED [INFO] flink-dist ......................................... SKIPPED [INFO] flink-fs-tests ..................................... SKIPPED [INFO] flink-yarn-tests ................................... SKIPPED [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ [INFO] Total time: 25:31 min [INFO] Finished at: 2017-03-06T12:07:47+00:00 [INFO] Final Memory: 161M/493M [INFO] ------------------------------------------------------------------------ [ERROR] Failed to execute goal org.apache.maven.plugins:maven-surefire-plugin:2.18.1:test (integration-tests) on project flink-connector-cassandra_2.10: There are test failures. [ERROR] [ERROR] Please refer to /home/travis/build/apache/flink/flink-connectors/flink-connector-cassandra/target/surefire-reports for the individual test results. [ERROR] -> [Help 1] [ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch. [ERROR] Re-run Maven using the -X switch to enable full debug logging. [ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles: [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException [ERROR] [ERROR] After correcting the problems, you can resume the build with the command [ERROR] mvn <goals> -rf :flink-connector-cassandra_2.10 Trying to KILL watchdog (1345). ./tools/travis_mvn_watchdog.sh: line 210: 1345 Terminated watchdog MVN exited with EXIT CODE: 1. java.io.FileNotFoundException: build-target/lib/flink-dist-*.jar (No such file or directory) at java.util.zip.ZipFile.open(Native Method) at java.util.zip.ZipFile.<init>(ZipFile.java:220) at java.util.zip.ZipFile.<init>(ZipFile.java:150) at java.util.zip.ZipFile.<init>(ZipFile.java:121) at sun.tools.jar.Main.list(Main.java:1060) at sun.tools.jar.Main.run(Main.java:291) at sun.tools.jar.Main.main(Main.java:1233) find: `./flink-yarn-tests/target/flink-yarn-tests*': No such file or directory
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3459#discussion_r104283268

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/StreamAggregator.java —
          @@ -0,0 +1,23 @@
          +package org.apache.flink.table.plan.nodes.datastream.aggs;
          — End diff –

          have no apache licence header. Rat check failure.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3459#discussion_r104283268 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/StreamAggregator.java — @@ -0,0 +1,23 @@ +package org.apache.flink.table.plan.nodes.datastream.aggs; — End diff – have no apache licence header. Rat check failure.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3459#discussion_r104282413

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          +
          + private LogicalWindow windowReference;
          + private String description;
          +
          + public DataStreamProcTimeTimeAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
          + RelDataType rowType, String description, LogicalWindow windowReference)

          { + super(cluster, traitSet, input); + + this.rowType = rowType; + this.description = description; + this.windowReference = windowReference; + + }

          +
          + @Override
          + protected RelDataType deriveRowType()

          { + // TODO Auto-generated method stub + 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 DataStreamProcTimeTimeAggregate(getCluster(), traitSet, inputs.get(0), getRowType(),
          + getDescription(), windowReference);
          +
          + }
          +
          + @Override
          + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv) {
          +
          + // Get the general parameters related to the datastream, inputs, result
          + TableConfig config = tableEnv.getConfig();
          — End diff –

          `config` is not referenced

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3459#discussion_r104282413 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { + + private LogicalWindow windowReference; + private String description; + + public DataStreamProcTimeTimeAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + RelDataType rowType, String description, LogicalWindow windowReference) { + super(cluster, traitSet, input); + + this.rowType = rowType; + this.description = description; + this.windowReference = windowReference; + + } + + @Override + protected RelDataType deriveRowType() { + // TODO Auto-generated method stub + 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 DataStreamProcTimeTimeAggregate(getCluster(), traitSet, inputs.get(0), getRowType(), + getDescription(), windowReference); + + } + + @Override + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv) { + + // Get the general parameters related to the datastream, inputs, result + TableConfig config = tableEnv.getConfig(); — End diff – `config` is not referenced
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3459#discussion_r104282451

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          — End diff –

          class annotation shouldn't be omit. Better description is need.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3459#discussion_r104282451 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { — End diff – class annotation shouldn't be omit. Better description is need.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3459#discussion_r104282631

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          +
          + private LogicalWindow windowReference;
          + private String description;
          +
          + public DataStreamProcTimeTimeAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
          + RelDataType rowType, String description, LogicalWindow windowReference)

          { + super(cluster, traitSet, input); + + this.rowType = rowType; + this.description = description; + this.windowReference = windowReference; + + }

          +
          + @Override
          + protected RelDataType deriveRowType()

          { + // TODO Auto-generated method stub + 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 DataStreamProcTimeTimeAggregate(getCluster(), traitSet, inputs.get(0), getRowType(),
          + getDescription(), windowReference);
          +
          + }
          +
          + @Override
          + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv) {
          +
          + // Get the general parameters related to the datastream, inputs, result
          + TableConfig config = tableEnv.getConfig();
          +
          + DataStream<Row> inputDataStream = ((DataStreamRel) getInput()).translateToPlan(tableEnv);
          +
          + TypeInformation<?>[] rowType = new TypeInformation<?>[getRowType().getFieldList().size()];
          — End diff –

          1. `getRowType().getFieldList()` can be reused, it's better to declare a new variable.
          2. `getRowType().getFieldList()` shows that `getRowType()` mustn't be null, also `getFieldList()` must greater than -1. So, here should guarantee such precondition first.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3459#discussion_r104282631 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { + + private LogicalWindow windowReference; + private String description; + + public DataStreamProcTimeTimeAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + RelDataType rowType, String description, LogicalWindow windowReference) { + super(cluster, traitSet, input); + + this.rowType = rowType; + this.description = description; + this.windowReference = windowReference; + + } + + @Override + protected RelDataType deriveRowType() { + // TODO Auto-generated method stub + 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 DataStreamProcTimeTimeAggregate(getCluster(), traitSet, inputs.get(0), getRowType(), + getDescription(), windowReference); + + } + + @Override + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv) { + + // Get the general parameters related to the datastream, inputs, result + TableConfig config = tableEnv.getConfig(); + + DataStream<Row> inputDataStream = ((DataStreamRel) getInput()).translateToPlan(tableEnv); + + TypeInformation<?>[] rowType = new TypeInformation<?> [getRowType().getFieldList().size()] ; — End diff – 1. `getRowType().getFieldList()` can be reused, it's better to declare a new variable. 2. `getRowType().getFieldList()` shows that `getRowType()` mustn't be null, also `getFieldList()` must greater than -1. So, here should guarantee such precondition first.
          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/3459#discussion_r104428922

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          +
          + private LogicalWindow windowReference;
          + private String description;
          +
          + public DataStreamProcTimeTimeAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
          + RelDataType rowType, String description, LogicalWindow windowReference)

          { + super(cluster, traitSet, input); + + this.rowType = rowType; + this.description = description; + this.windowReference = windowReference; + + }

          +
          + @Override
          + protected RelDataType deriveRowType()

          { + // TODO Auto-generated method stub + 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 DataStreamProcTimeTimeAggregate(getCluster(), traitSet, inputs.get(0), getRowType(),
          + getDescription(), windowReference);
          +
          + }
          +
          + @Override
          + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv) {
          +
          + // Get the general parameters related to the datastream, inputs, result
          + TableConfig config = tableEnv.getConfig();
          +
          + DataStream<Row> inputDataStream = ((DataStreamRel) getInput()).translateToPlan(tableEnv);
          +
          + TypeInformation<?>[] rowType = new TypeInformation<?>[getRowType().getFieldList().size()];
          — End diff –

          Not sure how you suggest to make the checks? Is there a style in flink for doing this?

          Also related to the suggestion of using a variable for getRowType().getFieldList() - I will do it, but in theory modern day compilers such as JAVA should identify and optimize this kind of things. And therefore I would expect the result is the same.

          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/3459#discussion_r104428922 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { + + private LogicalWindow windowReference; + private String description; + + public DataStreamProcTimeTimeAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + RelDataType rowType, String description, LogicalWindow windowReference) { + super(cluster, traitSet, input); + + this.rowType = rowType; + this.description = description; + this.windowReference = windowReference; + + } + + @Override + protected RelDataType deriveRowType() { + // TODO Auto-generated method stub + 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 DataStreamProcTimeTimeAggregate(getCluster(), traitSet, inputs.get(0), getRowType(), + getDescription(), windowReference); + + } + + @Override + public DataStream<Row> translateToPlan(StreamTableEnvironment tableEnv) { + + // Get the general parameters related to the datastream, inputs, result + TableConfig config = tableEnv.getConfig(); + + DataStream<Row> inputDataStream = ((DataStreamRel) getInput()).translateToPlan(tableEnv); + + TypeInformation<?>[] rowType = new TypeInformation<?> [getRowType().getFieldList().size()] ; — End diff – Not sure how you suggest to make the checks? Is there a style in flink for doing this? Also related to the suggestion of using a variable for getRowType().getFieldList() - I will do it, but in theory modern day compilers such as JAVA should identify and optimize this kind of things. And therefore I would expect the result is the same.
          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/3459#discussion_r104429058

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/StreamAggregator.java —
          @@ -0,0 +1,23 @@
          +package org.apache.flink.table.plan.nodes.datastream.aggs;
          — End diff –

          I added this before...indeed it was failing the RAT

          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/3459#discussion_r104429058 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/aggs/StreamAggregator.java — @@ -0,0 +1,23 @@ +package org.apache.flink.table.plan.nodes.datastream.aggs; — End diff – I added this before...indeed it was failing the RAT
          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/3459#discussion_r104430878

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          — End diff –

          I added a similar description with the other SQL related classes

          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/3459#discussion_r104430878 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { — End diff – I added a similar description with the other SQL related classes
          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/3459#discussion_r104431103

          — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java —
          @@ -0,0 +1,162 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow;
          +import org.apache.calcite.rel.type.RelDataType;
          +import org.apache.calcite.rel.type.RelDataTypeField;
          +import org.apache.flink.api.common.typeinfo.TypeInformation;
          +import org.apache.flink.api.java.typeutils.RowTypeInfo;
          +import org.apache.flink.streaming.api.datastream.DataStream;
          +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
          +import org.apache.flink.streaming.api.watermark.Watermark;
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
          +import org.apache.flink.streaming.api.windowing.time.Time;
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
          +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.DataStreamRel;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction;
          +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction;
          +import org.apache.flink.types.Row;
          +
          +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava {
          — End diff –

          what annotation? I am not familiar with the class annotation of Flink. is it Internal? Public? Thanks a lot for the clarification. Or were you referring to the documentation?

          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/3459#discussion_r104431103 — Diff: flink-libraries/flink-table/src/main/java/org/apache/flink/table/plan/nodes/datastream/DataStreamProcTimeTimeAggregate.java — @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.logical.LogicalWindow; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; +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.DataStreamRel; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateGlobalWindowFunction; +import org.apache.flink.table.plan.nodes.datastream.function.DataStreamProcTimeAggregateWindowFunction; +import org.apache.flink.types.Row; + +public class DataStreamProcTimeTimeAggregate extends DataStreamRelJava { — End diff – what annotation? I am not familiar with the class annotation of Flink. is it Internal? Public? Thanks a lot for the clarification. Or were you referring to the documentation?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske @wuchong @twalthr can you please take a look to see if you can merge this. Thanks

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3459 @fhueske @wuchong @twalthr can you please take a look to see if you can merge this. Thanks
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @huawei-flink, sorry for the long delay. I focused on adding support for retraction to the new UDAGG interface (PR #3470), support for time mode functions (PR #3370, #3425), and the most simple OVER window case (PR #3397) first. These additions should serve as a good basis for the more advanced OVER window cases.

          I had a look this PR and noticed a few things:

          • The Table API is implemented in Scala. We have a few Java classes (code copied & fixed from Calcite, Java examples, and tests) but do not intend to add more. The core parts like operators, translation, and runtime code have to be implemented in Scala. Mixing languages makes the maintenance more difficult.
          • Please squash your commits. Usually, PRs are opened with a single commit and new commits are added when feedback is addressed. Before merging a committer squashes these commits. However, it is too much effort to squash more than 50 commits including merge commits which can cause trouble. If multiple contributors worked on a PR, figure out how you can separate the work into one commit per author.
          • We have recently reworked our aggregation interface which will also serve as the interface for user-defined aggregations which should also be usable in OVER windows. Please use these aggregation functions. When designing the interface we had their use in OVER windows in mind. If you find that the interface is lacking a method, please start a discussion. However, we cannot have several incompatible aggregation interfaces in the Table API / SQL. Please rebase to the current master and use the new aggregation functions.
          • A couple of days ago, we added `PROCTIME()` and `ROWTIME()` methods which should be used to identify the time mode.
          • The first OVER window aggregation should serve as a blueprint for future OVER window implementations. We should try to keep the implementations as close as possible to share common code and make the overall maintenance of the code easier.

          Minor comments:

          • Why did you check modifications to the `.gitignore` files?
          • Please do not remove tests without replacing them with equivalent tests.
          • Please do not reformat classes (see FunctionCatalog). These changes cause additional effort when reviewing a PR.
          • Do not change ScalaDoc comments to JavaDoc comments in Scala code.
            ScalaDoc:
            ```
            /**
          • Comment
            */
            ```
            JavaDoc:
            ```
            /**
          • Comment
            */
            ```

          Thanks, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3459 Hi @huawei-flink, sorry for the long delay. I focused on adding support for retraction to the new UDAGG interface (PR #3470), support for time mode functions (PR #3370, #3425), and the most simple OVER window case (PR #3397) first. These additions should serve as a good basis for the more advanced OVER window cases. I had a look this PR and noticed a few things: The Table API is implemented in Scala. We have a few Java classes (code copied & fixed from Calcite, Java examples, and tests) but do not intend to add more. The core parts like operators, translation, and runtime code have to be implemented in Scala. Mixing languages makes the maintenance more difficult. Please squash your commits. Usually, PRs are opened with a single commit and new commits are added when feedback is addressed. Before merging a committer squashes these commits. However, it is too much effort to squash more than 50 commits including merge commits which can cause trouble. If multiple contributors worked on a PR, figure out how you can separate the work into one commit per author. We have recently reworked our aggregation interface which will also serve as the interface for user-defined aggregations which should also be usable in OVER windows. Please use these aggregation functions. When designing the interface we had their use in OVER windows in mind. If you find that the interface is lacking a method, please start a discussion. However, we cannot have several incompatible aggregation interfaces in the Table API / SQL. Please rebase to the current master and use the new aggregation functions. A couple of days ago, we added `PROCTIME()` and `ROWTIME()` methods which should be used to identify the time mode. The first OVER window aggregation should serve as a blueprint for future OVER window implementations. We should try to keep the implementations as close as possible to share common code and make the overall maintenance of the code easier. Minor comments: Why did you check modifications to the `.gitignore` files? Please do not remove tests without replacing them with equivalent tests. Please do not reformat classes (see FunctionCatalog). These changes cause additional effort when reviewing a PR. Do not change ScalaDoc comments to JavaDoc comments in Scala code. ScalaDoc: ``` /** Comment */ ``` JavaDoc: ``` /** Comment */ ``` Thanks, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          Hi @fhueske

          I will start with the minor comments: in principle those were done by mistake within the rebase...
          Regarding the FunctionCatalog - that was because initially we used our own implementation fro ProcTime()....the file as it is now it should not be modified. - now after the rebase the proctime is used

          Related to the main comments:
          1) scala/java - I think it is a bit restrictive and unfair to say that only scala is acceptable here for maintenance reasons given that a large part of the Flink project is written in java... Will you actually impose these restriction?
          2) related to the squash commits...we can check out again a new branch and add only the modifications in one push - I guess this should be ok?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3459 Hi @fhueske I will start with the minor comments: in principle those were done by mistake within the rebase... Regarding the FunctionCatalog - that was because initially we used our own implementation fro ProcTime()....the file as it is now it should not be modified. - now after the rebase the proctime is used Related to the main comments: 1) scala/java - I think it is a bit restrictive and unfair to say that only scala is acceptable here for maintenance reasons given that a large part of the Flink project is written in java... Will you actually impose these restriction? 2) related to the squash commits...we can check out again a new branch and add only the modifications in one push - I guess this should be ok?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user huawei-flink commented on the issue:

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

          @fhueske
          3. "•The first OVER window aggregation should serve as a blueprint for future OVER window implementations." - is this a general thought or you indicate that we need to rework the code based on some specific class?

          Show
          githubbot ASF GitHub Bot added a comment - Github user huawei-flink commented on the issue: https://github.com/apache/flink/pull/3459 @fhueske 3. "•The first OVER window aggregation should serve as a blueprint for future OVER window implementations." - is this a general thought or you indicate that we need to rework the code based on some specific class?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user fhueske commented on the issue:

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

          Hi @huawei-flink,

          most Maven modules in Flink are implemented in either Java or Scala. We have a few modules (like flink-runtime) which are mixed but didn't make good experience with this design. Also, there is a clear separation in these modules which components are implemented in Java and Scala. I think implementing the flink-table module in a mix of Java and Scala is not a good idea because it makes the maintenance more difficult than sticking to a single language.

          Regarding the squashing. I think the best approach is to create a new branch based on the current master and manually copying over those files that you would like to keep. Most of the code should be new files, so this should not be too much effort. By using the first OVER window as a blueprint I meant that you should try to follow a similar approach by using the same utils and a similar design. IMO, it makes sense if similar features share as much code as possible and follow the same design.

          Best, Fabian

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3459 Hi @huawei-flink, most Maven modules in Flink are implemented in either Java or Scala. We have a few modules (like flink-runtime) which are mixed but didn't make good experience with this design. Also, there is a clear separation in these modules which components are implemented in Java and Scala. I think implementing the flink-table module in a mix of Java and Scala is not a good idea because it makes the maintenance more difficult than sticking to a single language. Regarding the squashing. I think the best approach is to create a new branch based on the current master and manually copying over those files that you would like to keep. Most of the code should be new files, so this should not be too much effort. By using the first OVER window as a blueprint I meant that you should try to follow a similar approach by using the same utils and a similar design. IMO, it makes sense if similar features share as much code as possible and follow the same design. Best, Fabian
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user rtudoran opened a pull request:

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

          FLINK-5654 - Add processing time OVER RANGE 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-5654re

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

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


          commit b5eb64dc6f6b1036a3f6017ba1941f45347f2b67
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-03-15T12:58:20Z

          intermediate state

          commit ba1d7f91715e16b562177935db574f4ab8efb87c
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-03-15T13:08:27Z

          Merge branch 'master' of https://github.com/apache/flink into
          FLINK-5654re

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

          commit c1b806486a80a87052cda9629454bee0c12239de
          Author: rtudoran <tudoranradu@ymail.com>
          Date: 2017-03-15T16:23:29Z

          Implementation in Scala for JIRA issue 5654 (aggregates over time
          bounded windows working on processing time)


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user rtudoran opened a pull request: https://github.com/apache/flink/pull/3550 FLINK-5654 - Add processing time OVER RANGE 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-5654 re Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3550.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 #3550 commit b5eb64dc6f6b1036a3f6017ba1941f45347f2b67 Author: rtudoran <tudoranradu@ymail.com> Date: 2017-03-15T12:58:20Z intermediate state commit ba1d7f91715e16b562177935db574f4ab8efb87c Author: rtudoran <tudoranradu@ymail.com> Date: 2017-03-15T13:08:27Z Merge branch 'master' of https://github.com/apache/flink into FLINK-5654 re Conflicts: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala commit c1b806486a80a87052cda9629454bee0c12239de Author: rtudoran <tudoranradu@ymail.com> Date: 2017-03-15T16:23:29Z Implementation in Scala for JIRA issue 5654 (aggregates over time bounded windows working on processing time)
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @fhueske @shaoxuan-wang @twalthr @wuchong @shijinkui
          I re-implemented the time bounded window aggregates in scala. Can you please take a look and merge it. Thanks

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @fhueske @shaoxuan-wang @twalthr @wuchong @shijinkui I re-implemented the time bounded window aggregates in scala. Can you please take a look and merge it. Thanks
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106396314

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +
          +
          + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
          +
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          everytime it should create new accumulator, so i think it is no need to create on open method

          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/3550#discussion_r106396314 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – everytime it should create new accumulator, so i think it is no need to create on open method
          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/3550#discussion_r106395452

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          + .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
          +
          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time). Therefore the
          + // current system time is assign as the timestamp of the event to be
          + // recognize by the evictor
          +
          + val inputDataStreamTimed = inputDS
          + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDataStreamTimed.keyBy(0)
          — End diff –

          It seems should be inputDataStreamTimed.keyBy(partitionKeys:_*)

          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/3550#discussion_r106395452 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf [java.math.BigDecimal] .longValue() + + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). Therefore the + // current system time is assign as the timestamp of the event to be + // recognize by the evictor + + val inputDataStreamTimed = inputDS + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor()) + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDataStreamTimed.keyBy(0) — End diff – It seems should be inputDataStreamTimed.keyBy(partitionKeys:_*)
          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/3550#discussion_r106405078

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          + .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
          +
          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time). Therefore the
          + // current system time is assign as the timestamp of the event to be
          + // recognize by the evictor
          +
          + val inputDataStreamTimed = inputDS
          + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDataStreamTimed.keyBy(0)
          — End diff –

          Indeed - I missed-saw that ...and forgot to change it
          Thank you

          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/3550#discussion_r106405078 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf [java.math.BigDecimal] .longValue() + + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). Therefore the + // current system time is assign as the timestamp of the event to be + // recognize by the evictor + + val inputDataStreamTimed = inputDS + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor()) + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDataStreamTimed.keyBy(0) — End diff – Indeed - I missed-saw that ...and forgot to change it Thank you
          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/3550#discussion_r106405327

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +
          +
          + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
          +
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          indeed. this is more for convenience to have a concise code to access the accumulators

          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/3550#discussion_r106405327 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – indeed. this is more for convenience to have a concise code to access the accumulators
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          I have updated the code based on the review. @hongyuhong - thank you very much for the review
          @fhueske @twalthr - can you take a look and check if you can merge it please

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 I have updated the code based on the review. @hongyuhong - thank you very much for the review @fhueske @twalthr - can you take a look and check if you can merge it please
          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/3550#discussion_r106453475

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +
          +
          + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
          +
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          I will apply this fix also in my function

          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/3550#discussion_r106453475 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – I will apply this fix also in my function
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106600097

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          — End diff –

          `val time_boundary = logicWindow.constants.get(lowerboundIndex)`
          trim the two space, like
          `val time_boundary = logicWindow.constants.get(lowerboundIndex)`

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106600097 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) — End diff – `val time_boundary = logicWindow.constants.get(lowerboundIndex)` trim the two space, like `val time_boundary = logicWindow.constants.get(lowerboundIndex)`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106599719

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          — End diff –

          `def createTimeBoundedProcessingTimeOverWindow(` in one line.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106599719 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( — End diff – `def createTimeBoundedProcessingTimeOverWindow(` in one line.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106605103

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

          1. scaladoc is not well format, need a clear and detail scaladoc
          2. the class should be format first
          3. delete the un-used import phrase, such as `import org.apache.flink.api.java.typeutils.RowTypeInfo`

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106605103 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + — End diff – 1. scaladoc is not well format, need a clear and detail scaladoc 2. the class should be format first 3. delete the un-used import phrase, such as `import org.apache.flink.api.java.typeutils.RowTypeInfo`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106600464

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          + .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
          — End diff –

          `getValue2` returns `Comparable` value, but the value must be `java.math.BigDecimal`?
          Using match/case to guarantee its type.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106600464 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf [java.math.BigDecimal] .longValue() — End diff – `getValue2` returns `Comparable` value, but the value must be `java.math.BigDecimal`? Using match/case to guarantee its type.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106602970

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          — End diff –

          `aggregates` is array, also we can convert it to a `iterator`, then the `i` will be no needed. Iterator is more safe.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106602970 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 — End diff – `aggregates` is array, also we can convert it to a `iterator`, then the `i` will be no needed. Iterator is more safe.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106411491

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -136,13 +229,13 @@ class DataStreamOverAggregate(
          namedAggregates,
          inputType)

          • inputDS
            + inputDS
            .keyBy(partitionKeys: _*)
            .process(processFunction)
            .returns(rowTypeInfo)
            .name(aggOpName)
            .asInstanceOf[DataStream[Row]]
          • }
            + } // global non-partitioned aggregation
              • End diff –

          annotation should before `inputDS`

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106411491 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -136,13 +229,13 @@ class DataStreamOverAggregate( namedAggregates, inputType) inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + } // global non-partitioned aggregation End diff – annotation should before `inputDS`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106604502

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          — End diff –

          1. this class should be format first.
          2. IMO, all the `asInstanceOf` should be guarantee by the match/case except the object is determinated.
          3. `Calculate aggregated values output by aggregate buffer, and set them into output` should end with `.`, and the `@param` tag should be add
          4. the variable `i` should have clear name. At the same time `i` is used in three loop, that make the code hard to read.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106604502 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 — End diff – 1. this class should be format first. 2. IMO, all the `asInstanceOf` should be guarantee by the match/case except the object is determinated. 3. `Calculate aggregated values output by aggregate buffer, and set them into output` should end with `.`, and the `@param` tag should be add 4. the variable `i` should have clear name. At the same time `i` is used in three loop, that make the code hard to read.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106602209

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          — End diff –

          1. `DataStreamIncrementalAggregateWindowFunction` class is different from this class name.
          2. scaladoc `numGroupingKey`, `numAggregates`, 'finalRowArity' didn't exist
          3. IMO, all the function in the aggregate package have no enough clear doc to describe what/how and the key point function. package `aggregate` is lazy working.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106602209 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( — End diff – 1. `DataStreamIncrementalAggregateWindowFunction` class is different from this class name. 2. scaladoc `numGroupingKey`, `numAggregates`, 'finalRowArity' didn't exist 3. IMO, all the function in the aggregate package have no enough clear doc to describe what/how and the key point function. package `aggregate` is lazy working.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106601075

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          Using [flink] make this function can be accessed, that can work. But we need think one thing: whether there function need be visit in the `flink` range of package.
          My suggestion is `private[table]` to apply other function in this class.
          What other reviewer think about that?

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106601075 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – Using [flink] make this function can be accessed, that can work. But we need think one thing: whether there function need be visit in the `flink` range of package. My suggestion is `private [table] ` to apply other function in this class. What other reviewer think about that?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106599975

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          — End diff –

          if no used, delete the comment code lines

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106599975 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = — End diff – if no used, delete the comment code lines
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106605456

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

          `private val ` should be choose if the `aggregates` need not be access directly. Same to the class field.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106605456 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], — End diff – `private val ` should be choose if the `aggregates` need not be access directly. Same to the class field.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106605509

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +
          +
          + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
          +
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

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

          + var reuse:Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          + i = 0
          + while (i < forwardedFieldCount)

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

          +
          + //set the values of the result with the accumulators
          + i = 0
          + while (i < aggregates.length)

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

          +
          + out.collect(output)
          +
          — End diff –

          need not new line.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106605509 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + var reuse:Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, reuse.getField(i)) + i += 1 + } + + //set the values of the result with the accumulators + i = 0 + while (i < aggregates.length) { + val index = forwardedFieldCount + i + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + output.setField(index, aggregates(i).getValue(accumulator)) + i += 1 + } + + out.collect(output) + — End diff – need not new line.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106411557

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -191,3 +287,31 @@ class DataStreamOverAggregate(

          }

          +object DataStreamProcTimeCase {
          + class ProcTimeTimestampExtractor
          + extends AssignerWithPunctuatedWatermarks[Row] {
          +
          + override def checkAndGetNextWatermark(
          + lastElement: Row,
          + extractedTimestamp: Long): Watermark =

          { + null + }

          +
          + override def extractTimestamp(
          + element: Row,
          + previousElementTimestamp: Long): Long =

          { + System.currentTimeMillis() + }

          + }
          + /*
          + class MyWindowFunction extends AllWindowFunction[Row, Row, GlobalWindow] {
          — End diff –

          delete it if useless

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106411557 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -191,3 +287,31 @@ class DataStreamOverAggregate( } +object DataStreamProcTimeCase { + class ProcTimeTimestampExtractor + extends AssignerWithPunctuatedWatermarks [Row] { + + override def checkAndGetNextWatermark( + lastElement: Row, + extractedTimestamp: Long): Watermark = { + null + } + + override def extractTimestamp( + element: Row, + previousElementTimestamp: Long): Long = { + System.currentTimeMillis() + } + } + /* + class MyWindowFunction extends AllWindowFunction [Row, Row, GlobalWindow] { — End diff – delete it if useless
          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/3550#discussion_r106611866

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          — End diff –

          I am not sure what formatting should be done with respect to this. The method is split in 2 lines in order not to break the 100 characters per line style...
          Was done automatically by IDE formater

          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/3550#discussion_r106611866 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( — End diff – I am not sure what formatting should be done with respect to this. The method is split in 2 lines in order not to break the 100 characters per line style... Was done automatically by IDE formater
          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/3550#discussion_r106613249

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -136,13 +229,13 @@ class DataStreamOverAggregate(
          namedAggregates,
          inputType)

          • inputDS
            + inputDS
            .keyBy(partitionKeys: _*)
            .process(processFunction)
            .returns(rowTypeInfo)
            .name(aggOpName)
            .asInstanceOf[DataStream[Row]]
          • }
            + } // global non-partitioned aggregation
              • End diff –

          I agree...but that was from the code that already existed...from the previous merges.

          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/3550#discussion_r106613249 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -136,13 +229,13 @@ class DataStreamOverAggregate( namedAggregates, inputType) inputDS + inputDS .keyBy(partitionKeys: _*) .process(processFunction) .returns(rowTypeInfo) .name(aggOpName) .asInstanceOf[DataStream [Row] ] } + } // global non-partitioned aggregation End diff – I agree...but that was from the code that already existed...from the previous merges.
          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/3550#discussion_r106613551

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          agree
          @fhueske - what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106613551 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – agree @fhueske - what do you think?
          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/3550#discussion_r106613652

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          agree
          @fhueske - what do you think?
          I also used the model from other such functions

          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/3550#discussion_r106613652 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – agree @fhueske - what do you think? I also used the model from other such functions
          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/3550#discussion_r106614204

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          — End diff –

          my mistake...when I used the previous one as a model.

          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/3550#discussion_r106614204 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( — End diff – my mistake...when I used the previous one as a model.
          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/3550#discussion_r106614631

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          — End diff –

          in general I agree. but on the other hand it just creates additional objects and overhead. Considering that it is in a very specific case - I prefer to use it like this

          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/3550#discussion_r106614631 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 — End diff – in general I agree. but on the other hand it just creates additional objects and overhead. Considering that it is in a very specific case - I prefer to use it like this
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106617241

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          — End diff –

          93 characters, not reach 100. Intellij IDEA can see the result clearly.

          Show
          githubbot ASF GitHub Bot added a comment - Github user shijinkui commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106617241 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( — End diff – 93 characters, not reach 100. Intellij IDEA can see the result clearly.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @shijinkui thank you for the remarks. I have addressed them

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @shijinkui thank you for the remarks. I have addressed them
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @fhueske - can you please take a look over

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @fhueske - can you please take a look over
          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/3550#discussion_r106547382

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +
          +
          + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
          +
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          — End diff –

          Actually, accumulators can be created once in `open()` and be reused after the have been reset with `AggregateFunction.resetAccumulator()`

          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/3550#discussion_r106547382 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) — End diff – Actually, accumulators can be created once in `open()` and be reused after the have been reset with `AggregateFunction.resetAccumulator()`
          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/3550#discussion_r106545668

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          + .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
          +
          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time). Therefore the
          + // current system time is assign as the timestamp of the event to be
          + // recognize by the evictor
          +
          + val inputDataStreamTimed = inputDS
          — End diff –

          We should not assign processing timestamps as event timestamps because it overrides existing timestamps.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106545668 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf [java.math.BigDecimal] .longValue() + + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). Therefore the + // current system time is assign as the timestamp of the event to be + // recognize by the evictor + + val inputDataStreamTimed = inputDS — End diff – We should not assign processing timestamps as event timestamps because it overrides existing timestamps.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106630541

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          We could also move the logic to create the processing function into `AggreagteUtil` like all other operators that have to deal with aggregations.

          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/3550#discussion_r106630541 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – We could also move the logic to create the processing function into `AggreagteUtil` like all other operators that have to deal with aggregations.
          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/3550#discussion_r106630745

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          — End diff –

          We can do that in `open()` because it is just called once. However, in the hot path we should use `while` loops because Scala `for` loops have significant overhead.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106630745 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 — End diff – We can do that in `open()` because it is just called once. However, in the hot path we should use `while` loops because Scala `for` loops have significant overhead.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106630898

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/DataStreamProcTimeBoundAggIntegrationITCase.scala —
          @@ -0,0 +1,191 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.apache.flink.types.Row
          +import org.junit.Assert._
          +import org.junit._
          +import scala.collection.mutable
          +
          +class DataStreamProcTimeBoundAggIntegrationITCase extends StreamingWithStateTestBase {
          — End diff –

          Please see the comment on PR #3547 regarding the tests.
          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/3550#discussion_r106630898 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/DataStreamProcTimeBoundAggIntegrationITCase.scala — @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.apache.flink.types.Row +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable + +class DataStreamProcTimeBoundAggIntegrationITCase extends StreamingWithStateTestBase { — End diff – Please see the comment on PR #3547 regarding the tests. 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/3550#discussion_r106546899

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -191,3 +287,31 @@ class DataStreamOverAggregate(

          }

          +object DataStreamProcTimeCase {
          + class ProcTimeTimestampExtractor
          + extends AssignerWithPunctuatedWatermarks[Row] {
          +
          + override def checkAndGetNextWatermark(
          + lastElement: Row,
          + extractedTimestamp: Long): Watermark =

          { + null + }

          +
          + override def extractTimestamp(
          + element: Row,
          + previousElementTimestamp: Long): Long = {
          + System.currentTimeMillis()
          — End diff –

          `System.currentTimeMillis()` is not not strictly increasing. For instance the time of a machine can be synced by a timeserver. So we should always remember the last emitted timestamp and return the `max` of the last and the current time.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106546899 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -191,3 +287,31 @@ class DataStreamOverAggregate( } +object DataStreamProcTimeCase { + class ProcTimeTimestampExtractor + extends AssignerWithPunctuatedWatermarks [Row] { + + override def checkAndGetNextWatermark( + lastElement: Row, + extractedTimestamp: Long): Watermark = { + null + } + + override def extractTimestamp( + element: Row, + previousElementTimestamp: Long): Long = { + System.currentTimeMillis() — End diff – `System.currentTimeMillis()` is not not strictly increasing. For instance the time of a machine can be synced by a timeserver. So we should always remember the last emitted timestamp and return the `max` of the last and the current time.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106629913

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,66 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + var time_boundary = 0L
          +
          + logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time). Therefore the
          + // current system time is assign as the timestamp of the event to be
          + // recognize by the evictor
          +
          + val inputDataStreamTimed = inputDS
          + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty)

          { + inputDataStreamTimed.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateWindowFunction[GlobalWindow] + (aggregates,aggFields,inputType.getFieldCount)) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + + }

          else {
          + inputDataStreamTimed.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          + .apply(new DataStreamProcTimeAggregateGlobalWindowFunction[GlobalWindow](
          — End diff –

          I think a `ProcessFunction` would be more efficient for this use case. With a `WindowFunction` and a `GlobalWindow`, we have to aggregate all rows for each emitted row. With a `ProcessFunction` we can just add the new row and retract those rows which are not included anymore.

          Moreover, we have to change the logic to a `ProcessFunction` anyway when we extend this to support `FOLLOWING`

          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/3550#discussion_r106629913 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,66 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + var time_boundary = 0L + + logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). Therefore the + // current system time is assign as the timestamp of the event to be + // recognize by the evictor + + val inputDataStreamTimed = inputDS + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor()) + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDataStreamTimed.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateWindowFunction[GlobalWindow] + (aggregates,aggFields,inputType.getFieldCount)) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + + } else { + inputDataStreamTimed.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateGlobalWindowFunction [GlobalWindow] ( — End diff – I think a `ProcessFunction` would be more efficient for this use case. With a `WindowFunction` and a `GlobalWindow`, we have to aggregate all rows for each emitted row. With a `ProcessFunction` we can just add the new row and retract those rows which are not included anymore. Moreover, we have to change the logic to a `ProcessFunction` anyway when we extend this to support `FOLLOWING`
          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/3550#discussion_r106546529

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -191,3 +287,31 @@ class DataStreamOverAggregate(

          }

          +object DataStreamProcTimeCase {
          + class ProcTimeTimestampExtractor
          — End diff –

          We should not use a `TimestampExtractor` to assign processing time timestamps as event timestamps.
          I think the `ProcTimeTimestampExtractor` does not need to be wrapped in an `object`.

          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/3550#discussion_r106546529 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -191,3 +287,31 @@ class DataStreamOverAggregate( } +object DataStreamProcTimeCase { + class ProcTimeTimestampExtractor — End diff – We should not use a `TimestampExtractor` to assign processing time timestamps as event timestamps. I think the `ProcTimeTimestampExtractor` does not need to be wrapped in an `object`.
          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/3550#discussion_r106548270

          — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/DataStreamProcTimeBoundAggIntegrationTest.scala —
          @@ -0,0 +1,191 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.apache.flink.types.Row
          +import org.junit.Assert._
          +import org.junit._
          +import scala.collection.mutable
          +
          +class DataStreamProcTimeBoundAggIntegrationTest extends StreamingWithStateTestBase {
          — End diff –

          Please see the comments on PR #3547 regarding the tests.

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r106548270 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/DataStreamProcTimeBoundAggIntegrationTest.scala — @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.apache.flink.types.Row +import org.junit.Assert._ +import org.junit._ +import scala.collection.mutable + +class DataStreamProcTimeBoundAggIntegrationTest extends StreamingWithStateTestBase { — End diff – Please see the comments on PR #3547 regarding the tests.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106630792

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,107 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          + var reuse:Row = null
          — 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/3550#discussion_r106630792 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + var reuse:Row = null — 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/3550#discussion_r106631183

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

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import java.util.

          { List => JList }

          — End diff –

          Can you revert the `import` changes?

          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/3550#discussion_r106631183 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,23 +17,54 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import java.util. { List => JList } — End diff – Can you revert the `import` changes?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r106545012

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          — End diff –

          Check 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/3550#discussion_r106545012 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) — End diff – Check 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/3550#discussion_r106629136

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,66 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + var time_boundary = 0L
          +
          + logicWindow.constants.get(lowerboundIndex).getValue2 match {
          — End diff –

          This can be changed to
          ```
          val timeBoundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) .getValue2.asInstanceOf[java.math.BigDecimal].longValue() case _ => throw new TableException("OVER Window boundaries must be numeric") }

          ```

          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/3550#discussion_r106629136 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,66 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + var time_boundary = 0L + + logicWindow.constants.get(lowerboundIndex).getValue2 match { — End diff – This can be changed to ``` val timeBoundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) .getValue2.asInstanceOf[java.math.BigDecimal].longValue() case _ => throw new TableException("OVER Window boundaries must be numeric") } ```
          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/3550#discussion_r106544854

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[IntervalSqlType]
          — End diff –

          We can explicitly check if a window is a ROW or a RANGE window by `overWindow.isRows`. I'm also not sure about the `overWindow.upperBound.isPreceding` condition.

          I think 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 RANGE 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/3550#discussion_r106544854 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,6 +137,10 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [IntervalSqlType] — End diff – We can explicitly check if a window is a ROW or a RANGE window by `overWindow.isRows`. I'm also not sure about the `overWindow.upperBound.isPreceding` condition. I think 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 RANGE 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/3550#discussion_r106546112

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,64 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + // final long time_boundary =
          + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + val time_boundary = logicWindow.constants.get(lowerboundIndex)
          + .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
          +
          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time). Therefore the
          + // current system time is assign as the timestamp of the event to be
          + // recognize by the evictor
          +
          + val inputDataStreamTimed = inputDS
          + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDataStreamTimed.keyBy(partitionKeys:_*)
          + .window(GlobalWindows.create())
          + .trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          + .apply(new DataStreamIncrementalAggregateWindowFunction[GlobalWindow]
          — End diff –

          Let's use a process function. We have to change the code anyway once we want to support `FOLLOWING`. Also a `ProcessFunction` does not need to aggregate all rows for each row but remember the accumulators of the last row, add the new row and retract the old ones.

          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/3550#discussion_r106546112 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,64 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + // final long time_boundary = + // Long.parseLong(windowReference.getConstants().get(1).getValue().toString()); + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + val time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf [java.math.BigDecimal] .longValue() + + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). Therefore the + // current system time is assign as the timestamp of the event to be + // recognize by the evictor + + val inputDataStreamTimed = inputDS + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor()) + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDataStreamTimed.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamIncrementalAggregateWindowFunction [GlobalWindow] — End diff – Let's use a process function. We have to change the code anyway once we want to support `FOLLOWING`. Also a `ProcessFunction` does not need to aggregate all rows for each row but remember the accumulators of the last row, add the new row and retract the old ones.
          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/3550#discussion_r106547467

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,104 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.api.java.typeutils.RowTypeInfo
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +
          +
          + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
          +
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

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

          + var reuse:Row = null
          — End diff –

          `var reuse: Row` +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/3550#discussion_r106547467 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.AggregateFunction + + + + //org.apache.flink.streaming.api.functions.windowing.AllWindowFunction + +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + var reuse:Row = null — End diff – `var reuse: Row` +space
          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/3550#discussion_r106658350

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          @fhueske I could do that - just let me know if this is what you want. For me it would definitely make sense

          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/3550#discussion_r106658350 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – @fhueske I could do that - just let me know if this is what you want. For me it would definitely make sense
          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/3550#discussion_r106658506

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param numGroupingKey The number of grouping keys.
          + * @param numAggregates The number of aggregates.
          + * @param finalRowArity The arity of the final output row.
          + */
          +class DataStreamIncrementalAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          — End diff –

          I will use than everywhere while

          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/3550#discussion_r106658506 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param numGroupingKey The number of grouping keys. + * @param numAggregates The number of aggregates. + * @param finalRowArity The arity of the final output row. + */ +class DataStreamIncrementalAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 — End diff – I will use than everywhere while
          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/3550#discussion_r106659524

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          else if (overWindow.lowerBound.getOffset.getType.isInstanceOf[IntervalSqlType]
          — End diff –

          @fhueske
          As a personal preference I like to have the conditions as explicit as possible just to make sure it will not trigger in undesirable cases. I did not knew about the isRows() function...of course I can check that.
          Regarding the upper bound my reasoning was that if we will support other constructs in the future -e.g. FOLLOWING, fixed time boundaries...whatever else will be invented... it is much easier to integrate those if we have this explicit check now..rather then searching for it at that moment
          However, if you prefer it removed of course it is not hard to do that

          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/3550#discussion_r106659524 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,6 +137,10 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } else if (overWindow.lowerBound.getOffset.getType.isInstanceOf [IntervalSqlType] — End diff – @fhueske As a personal preference I like to have the conditions as explicit as possible just to make sure it will not trigger in undesirable cases. I did not knew about the isRows() function...of course I can check that. Regarding the upper bound my reasoning was that if we will support other constructs in the future -e.g. FOLLOWING, fixed time boundaries...whatever else will be invented... it is much easier to integrate those if we have this explicit check now..rather then searching for it at that moment However, if you prefer it removed of course it is not hard to do that
          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/3550#discussion_r106659962

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,66 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + var time_boundary = 0L
          +
          + logicWindow.constants.get(lowerboundIndex).getValue2 match {
          — End diff –

          Thanks

          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/3550#discussion_r106659962 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,66 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + var time_boundary = 0L + + logicWindow.constants.get(lowerboundIndex).getValue2 match { — End diff – Thanks
          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/3550#discussion_r106666656

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +154,66 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(
          + 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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          + var time_boundary = 0L
          +
          + logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time). Therefore the
          + // current system time is assign as the timestamp of the event to be
          + // recognize by the evictor
          +
          + val inputDataStreamTimed = inputDS
          + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty)

          { + inputDataStreamTimed.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateWindowFunction[GlobalWindow] + (aggregates,aggFields,inputType.getFieldCount)) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + + }

          else {
          + inputDataStreamTimed.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          + .apply(new DataStreamProcTimeAggregateGlobalWindowFunction[GlobalWindow](
          — End diff –

          @fhueske
          If you are not strict with that I would like to keep the window based implementation. Particularly that once this is merged the next thing is that I want to create additional JIRA issues to support Distinct aggregates for this as well. Those for sure require having all data at once.
          Regarding the support for FOLLOWING - I do not see any problems there as I can of course work with the triggers to offer the same functionality (register the trigger to trigger x elements/time in the future).
          I do not know what capability support does processfunction provides for evicting events.
          I looked over the example
          https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/process_function.html
          and I do not see any eviction mechanism. This means there will be a lot of logic and overhead moved in identifying what needs to be evicted.
          All in all...I am not sure how this would look in ProcessFunction and I have some doubts that the end result is better. However, let me know what you think and whether you want it explicitly as a processfunction (case in which I would like to ask if you can please point me to some examples of how to manage eviction) or it can stay as it is.

          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/3550#discussion_r106666656 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +154,66 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow( + 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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + var time_boundary = 0L + + logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => time_boundary = logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). Therefore the + // current system time is assign as the timestamp of the event to be + // recognize by the evictor + + val inputDataStreamTimed = inputDS + .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor()) + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDataStreamTimed.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateWindowFunction[GlobalWindow] + (aggregates,aggFields,inputType.getFieldCount)) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + + } else { + inputDataStreamTimed.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateGlobalWindowFunction [GlobalWindow] ( — End diff – @fhueske If you are not strict with that I would like to keep the window based implementation. Particularly that once this is merged the next thing is that I want to create additional JIRA issues to support Distinct aggregates for this as well. Those for sure require having all data at once. Regarding the support for FOLLOWING - I do not see any problems there as I can of course work with the triggers to offer the same functionality (register the trigger to trigger x elements/time in the future). I do not know what capability support does processfunction provides for evicting events. I looked over the example https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/process_function.html and I do not see any eviction mechanism. This means there will be a lot of logic and overhead moved in identifying what needs to be evicted. All in all...I am not sure how this would look in ProcessFunction and I have some doubts that the end result is better. However, let me know what you think and whether you want it explicitly as a processfunction (case in which I would like to ask if you can please point me to some examples of how to manage eviction) or it can stay as it is.
          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/3550#discussion_r106667006

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -191,3 +287,31 @@ class DataStreamOverAggregate(

          }

          +object DataStreamProcTimeCase {
          + class ProcTimeTimestampExtractor
          + extends AssignerWithPunctuatedWatermarks[Row] {
          +
          + override def checkAndGetNextWatermark(
          + lastElement: Row,
          + extractedTimestamp: Long): Watermark =

          { + null + }

          +
          + override def extractTimestamp(
          + element: Row,
          + previousElementTimestamp: Long): Long = {
          + System.currentTimeMillis()
          — End diff –

          Good point! Thanks.

          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/3550#discussion_r106667006 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -191,3 +287,31 @@ class DataStreamOverAggregate( } +object DataStreamProcTimeCase { + class ProcTimeTimestampExtractor + extends AssignerWithPunctuatedWatermarks [Row] { + + override def checkAndGetNextWatermark( + lastElement: Row, + extractedTimestamp: Long): Watermark = { + null + } + + override def extractTimestamp( + element: Row, + previousElementTimestamp: Long): Long = { + System.currentTimeMillis() — End diff – Good point! Thanks.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @fhueske

          "It would also be good if you could remove the merge commit from the PR branch. Merge commits make it much harder to squash commits and merge them to the master branch."

          I am sorry but I am not sure what should I do exactly....
          Can you give me some pointers of what would you expect?
          ..The only option I know is to abandon this, recreate a new branch from current master, and move the code and redo the pull request...which I would expect is not the best option...nor what you expect

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @fhueske "It would also be good if you could remove the merge commit from the PR branch. Merge commits make it much harder to squash commits and merge them to the master branch." I am sorry but I am not sure what should I do exactly.... Can you give me some pointers of what would you expect? ..The only option I know is to abandon this, recreate a new branch from current master, and move the code and redo the pull request...which I would expect is not the best option...nor what you expect
          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/3550#discussion_r106669122

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,107 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          + var reuse:Row = null
          — End diff –

          not sure where...

          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/3550#discussion_r106669122 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + var reuse:Row = null — End diff – not sure where...
          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/3550#discussion_r106672193

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -191,3 +287,31 @@ class DataStreamOverAggregate(

          }

          +object DataStreamProcTimeCase {
          + class ProcTimeTimestampExtractor
          — End diff –

          @fhueske
          Given also your previous comment should I take this completely down?

          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/3550#discussion_r106672193 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -191,3 +287,31 @@ class DataStreamOverAggregate( } +object DataStreamProcTimeCase { + class ProcTimeTimestampExtractor — End diff – @fhueske Given also your previous comment should I take this completely down?
          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/3550#discussion_r106688280

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          It is not to much logic...and the overall "createTimeBoundedProcessingTimeOverWindow" is not bigger than the other method for unbound.
          ...as I said...it is the same...so as you prefer

          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/3550#discussion_r106688280 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – It is not to much logic...and the overall "createTimeBoundedProcessingTimeOverWindow" is not bigger than the other method for unbound. ...as I said...it is the same...so as you prefer
          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/3550#discussion_r106688499

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -191,3 +287,31 @@ class DataStreamOverAggregate(

          }

          +object DataStreamProcTimeCase {
          + class ProcTimeTimestampExtractor
          — End diff –

          @fhueske
          Given also your previous comment i would assume this goes down ...which i did

          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/3550#discussion_r106688499 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -191,3 +287,31 @@ class DataStreamOverAggregate( } +object DataStreamProcTimeCase { + class ProcTimeTimestampExtractor — End diff – @fhueske Given also your previous comment i would assume this goes down ...which i did
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @fhueske - Thank you very much for your review. It was very useful. I integrated and addressed most of the remarks you made.
          What remains to be discussed and it is not addressed is:
          -using the processingFunction vs keeping the Window-based implementation (i prefer and believe is more appropriate here later as mentioned)
          -doing the removal of the merge commit...as per my previous comment i do not know exactly what to do about that

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @fhueske - Thank you very much for your review. It was very useful. I integrated and addressed most of the remarks you made. What remains to be discussed and it is not addressed is: -using the processingFunction vs keeping the Window-based implementation (i prefer and believe is more appropriate here later as mentioned) -doing the removal of the merge commit...as per my previous comment i do not know exactly what to do about that
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @fhueske @twalthr - I am not sure if you saw the updates last Friday with all the things that are going on
          If you have a chance to look over it would be great - I would like to merge and close this pull reuqest

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @fhueske @twalthr - I am not sure if you saw the updates last Friday with all the things that are going on If you have a chance to look over it would be great - I would like to merge and close this pull reuqest
          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/3550#discussion_r107017957

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          — End diff –

          `lowerboundIndex` -> `lowerBoundIndex`

          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/3550#discussion_r107017957 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count — End diff – `lowerboundIndex` -> `lowerBoundIndex`
          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/3550#discussion_r107018893

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          — End diff –

          remove newline

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107018893 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + — End diff – remove newline
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r107019299

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          — End diff –

          remove one of two spaces between `=` and `AggregateUtil`

          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/3550#discussion_r107019299 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( — End diff – remove one of two spaces between `=` and `AggregateUtil`
          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/3550#discussion_r107018828

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match {
          + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex)
          — End diff –

          simplify to `case bd: java.math.BigDecimal => bd.longValue()`

          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/3550#discussion_r107018828 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) — End diff – simplify to `case bd: java.math.BigDecimal => bd.longValue()`
          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/3550#discussion_r107020183

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          — End diff –

          Please check comment

          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/3550#discussion_r107020183 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither — End diff – Please check comment
          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/3550#discussion_r107023400

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + var i = 0
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          + var reuse: Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          — End diff –

          why `if needed`?

          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/3550#discussion_r107023400 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + var i = 0 + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + var reuse: Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed — End diff – why `if needed`?
          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/3550#discussion_r107023458

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + //initialize the values of the aggregators by re-creating them
          — End diff –

          please update comment

          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/3550#discussion_r107023458 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + //initialize the values of the aggregators by re-creating them — End diff – please update comment
          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/3550#discussion_r107020630

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time).
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty)

          { + inputDS.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateWindowFunction[GlobalWindow] + (aggregates,aggFields,inputType.getFieldCount)) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + + }

          else {
          + inputDS.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          + .apply(new DataStreamProcTimeAggregateGlobalWindowFunction[GlobalWindow](
          + aggregates,aggFields,inputType.getFieldCount))
          + .setParallelism(1)
          — End diff –

          `windowAll` is always executed with parallelism 1

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107020630 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDS.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateWindowFunction[GlobalWindow] + (aggregates,aggFields,inputType.getFieldCount)) + .returns(rowTypeInfo) + .name(aggOpName) + .asInstanceOf[DataStream[Row]] + + } else { + inputDS.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) + .apply(new DataStreamProcTimeAggregateGlobalWindowFunction [GlobalWindow] ( + aggregates,aggFields,inputType.getFieldCount)) + .setParallelism(1) — End diff – `windowAll` is always executed with parallelism 1
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r107023660

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          + i = i + 1
          — End diff –

          `i += 1`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107023660 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) + i = i + 1 — End diff – `i += 1`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r107022916

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          — End diff –

          rm newline

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107022916 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + — End diff – rm newline
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r107017477

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

          -import org.apache.calcite.plan.

          {RelOptCluster, RelTraitSet}

          +import java.util.

          { List => JList }

          +
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
          +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.SingleRel
          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.core.Window
          +import org.apache.calcite.rel.core.Window.Group
          +import org.apache.calcite.sql.`type`.BasicSqlType
          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
          +import org.apache.flink.table.api.TableException
          import org.apache.flink.table.calcite.FlinkTypeFactory
          -import org.apache.flink.table.runtime.aggregate._
          +import org.apache.flink.table.functions.ProcTimeType
          +import org.apache.flink.table.functions.RowTimeType
          import org.apache.flink.table.plan.nodes.OverAggregate
          +import org.apache.flink.table.runtime.aggregate.AggregateUtil
          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 org.apache.calcite.sql.`type`.IntervalSqlType
          +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks
          +import org.apache.flink.streaming.api.watermark.Watermark
          +import org.apache.calcite.rex.RexInputRef
          +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor
          +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
          +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger
          — End diff –

          Please remove unused imports like `CountTrigger`

          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/3550#discussion_r107017477 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -17,23 +17,52 @@ */ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan. {RelOptCluster, RelTraitSet} +import java.util. { List => JList } + +import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair +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.SingleRel 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.core.Window +import org.apache.calcite.rel.core.Window.Group +import org.apache.calcite.sql.`type`.BasicSqlType 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 +import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.table.functions.ProcTimeType +import org.apache.flink.table.functions.RowTimeType import org.apache.flink.table.plan.nodes.OverAggregate +import org.apache.flink.table.runtime.aggregate.AggregateUtil 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 org.apache.calcite.sql.`type`.IntervalSqlType +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks +import org.apache.flink.streaming.api.watermark.Watermark +import org.apache.calcite.rex.RexInputRef +import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger — End diff – Please remove unused imports like `CountTrigger`
          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/3550#discussion_r107022806

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length) {
          + accumulators.setField(i, aggregates.createAccumulator())
          + i = i + 1
          — End diff –

          `i += 1`

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107022806 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates .createAccumulator()) + i = i + 1 — End diff – `i += 1`
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r107019094

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          — End diff –

          rm newline

          Show
          githubbot ASF GitHub Bot added a comment - Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107019094 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + — End diff – rm newline
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3550#discussion_r107026831

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala —
          @@ -785,7 +785,7 @@ object AggregateUtil

          { (propPos._1, propPos._2) }
          • private def transformToAggregateFunctions(
              • End diff –

          I'm OK with making this method `private[flink]`. @shijinkui is right, that this is more visible than necessary but on the other hand, many other methods of the Table API have been declared `private[flink]` as well.

          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/3550#discussion_r107026831 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala — @@ -785,7 +785,7 @@ object AggregateUtil { (propPos._1, propPos._2) } private def transformToAggregateFunctions( End diff – I'm OK with making this method `private [flink] `. @shijinkui is right, that this is more visible than necessary but on the other hand, many other methods of the Table API have been declared `private [flink] ` 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/3550#discussion_r107023802

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          +
          + var reuse:Row = null
          — 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/3550#discussion_r107023802 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + + var reuse:Row = null — 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/3550#discussion_r107025846

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

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testMaxAggregatation(): Unit = {
          — End diff –

          The aggregation functions are separately checked. We do not need tests that validate that each type of aggregation works with all types of aggregation functions.

          Moreover, the tests in this class do not execute any query but validate that the correct plan is produced. Please have a look at the other test methods and adapt your tests accordingly. I think two tests (partitioned and non-partitioned) with multiple aggregates should be sufficient.

          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/3550#discussion_r107025846 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +252,179 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testMaxAggregatation(): Unit = { — End diff – The aggregation functions are separately checked. We do not need tests that validate that each type of aggregation works with all types of aggregation functions. Moreover, the tests in this class do not execute any query but validate that the correct plan is produced. Please have a look at the other test methods and adapt your tests accordingly. I think two tests (partitioned and non-partitioned) with multiple aggregates should be sufficient.
          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/3550#discussion_r107023949

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          +
          + var reuse:Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          — End diff –

          why `if needed`?

          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/3550#discussion_r107023949 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + + var reuse:Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed — End diff – why `if needed`?
          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/3550#discussion_r107024511

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

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

          +
          +
          + @Test
          + def testAvgSumAggregatationPartition(): Unit = {
          +
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," +
          + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable"
          +
          + val t = StreamTestData.get5TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new ProcTimeTimestamp())
          — End diff –

          We should not require to set timestamps for a processing time 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/3550#discussion_r107024511 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +317,92 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + + @Test + def testAvgSumAggregatationPartition(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable" + + val t = StreamTestData.get5TupleDataStream(env) + .assignTimestampsAndWatermarks(new ProcTimeTimestamp()) — End diff – We should not require to set timestamps for a processing time 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/3550#discussion_r107017682

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          — End diff –

          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/3550#discussion_r107017682 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() — End diff – 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/3550#discussion_r107020368

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time).
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDS.keyBy(partitionKeys:_*)
          + .window(GlobalWindows.create())
          + .trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          — End diff –

          Doesn't the `TimeEvictor` evict based on event-time? How can we ensure processing time semantics with this evictor?

          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/3550#discussion_r107020368 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDS.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) — End diff – Doesn't the `TimeEvictor` evict based on event-time? How can we ensure processing time semantics with this evictor?
          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/3550#discussion_r107018019

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match {
          — End diff –

          `time_boundary` -> `timeBoundary`

          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/3550#discussion_r107018019 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { — End diff – `time_boundary` -> `timeBoundary`
          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/3550#discussion_r107023706

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          — End diff –

          update comment

          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/3550#discussion_r107023706 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + //initialize the values of the aggregators by re-creating them — End diff – update comment
          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/3550#discussion_r107022459

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          — End diff –

          We indent parameters with 4 and extends with 2 spaces

          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/3550#discussion_r107022459 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], — End diff – We indent parameters with 4 and extends with 2 spaces
          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/3550#discussion_r107024971

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

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

          +
          +
          + @Test
          + def testAvgSumAggregatationPartition(): Unit = {
          +
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," +
          + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable"
          +
          + val t = StreamTestData.get5TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new ProcTimeTimestamp())
          + .toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
          +
          + tEnv.registerTable("MyTable", t)
          +
          + val result = tEnv.sql(sqlQuery).toDataStream[Row]
          + result.addSink(new StreamITCase.StringSink)
          + env.execute()
          +
          + val expected = mutable.MutableList(
          — End diff –

          Processing time is hard to validate. This test does check the feature correctly. We should have a look at the processing time tests in the DataStream API and check if there are some tools or best practices that we can use.

          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/3550#discussion_r107024971 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +317,92 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + + @Test + def testAvgSumAggregatationPartition(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable" + + val t = StreamTestData.get5TupleDataStream(env) + .assignTimestampsAndWatermarks(new ProcTimeTimestamp()) + .toTable(tEnv).as('a, 'b, 'c, 'd, 'e) + + tEnv.registerTable("MyTable", t) + + val result = tEnv.sql(sqlQuery).toDataStream [Row] + result.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( — End diff – Processing time is hard to validate. This test does check the feature correctly. We should have a look at the processing time tests in the DataStream API and check if there are some tools or best practices that we can use.
          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/3550#discussion_r107017278

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

          { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + }

          else if (overWindow.lowerBound.isPreceding() && !overWindow.lowerBound.isUnbounded() &&
          — End diff –

          please remove the parentheses from the method calls without arguments: `overWindow.lowerBound.isPreceding()` -> `overWindow.lowerBound.isPreceding`

          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/3550#discussion_r107017278 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -106,6 +135,10 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS) + } else if (overWindow.lowerBound.isPreceding() && !overWindow.lowerBound.isUnbounded() && — End diff – please remove the parentheses from the method calls without arguments: `overWindow.lowerBound.isPreceding()` -> `overWindow.lowerBound.isPreceding`
          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/3550#discussion_r107019651

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          — End diff –

          add space
          `namedAggregates.map(_.getKey),inputType, needRetraction = false` ->
          `namedAggregates.map(_.getKey), inputType, needRetraction = false`

          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/3550#discussion_r107019651 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) — End diff – add space `namedAggregates.map(_.getKey),inputType, needRetraction = false` -> `namedAggregates.map(_.getKey), inputType, needRetraction = false`
          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/3550#discussion_r107068594

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          IMHO. we can using `ProcessFunction` implement this feature.

          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/3550#discussion_r107068594 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( — End diff – IMHO. we can using `ProcessFunction` implement this feature.
          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/3550#discussion_r107085313

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time).
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDS.keyBy(partitionKeys:_*)
          + .window(GlobalWindows.create())
          + .trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          — End diff –

          @fhueske Actually no - it works on the time that is set. Basically if the user operates on processing time than this is what it will use. It checks at the beginning of the eviction if the record has processing time and if so that it operates on it. I both verified it manually as well as with tests.

          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/3550#discussion_r107085313 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDS.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) — End diff – @fhueske Actually no - it works on the time that is set. Basically if the user operates on processing time than this is what it will use. It checks at the beginning of the eviction if the record has processing time and if so that it operates on it. I both verified it manually as well as with tests.
          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/3550#discussion_r107085563

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + var i = 0
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          + var reuse: Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          — End diff –

          @fhueske I would have said that it is always needed to carry on also the contents of the last event...but this is a too strong statement. Although i did not see examples where that is not the case...there are many SQL examples out there. Also, it might be that the optimizer of calcite changes at some point.

          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/3550#discussion_r107085563 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + var i = 0 + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + var reuse: Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed — End diff – @fhueske I would have said that it is always needed to carry on also the contents of the last event...but this is a too strong statement. Although i did not see examples where that is not the case...there are many SQL examples out there. Also, it might be that the optimizer of calcite changes at some point.
          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/3550#discussion_r107085614

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala —
          @@ -0,0 +1,108 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.flink.table.runtime.aggregate
          +
          +import java.lang.Iterable
          +
          +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.table.functions.Accumulator
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichWindowFunction[Row, Row, Tuple, W] {
          +
          +private var output: Row = _
          +private var accumulators: Row= _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + key: Tuple,
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          + var i = 0
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          +
          + var reuse:Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          — End diff –

          @fhueske -same as before

          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/3550#discussion_r107085614 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateWindowFunction.scala — @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.aggregate + +import java.lang.Iterable + +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.table.functions.Accumulator + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichWindowFunction [Row, Row, Tuple, W] { + +private var output: Row = _ +private var accumulators: Row= _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + key: Tuple, + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + var i = 0 + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + + var reuse:Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed — End diff – @fhueske -same as before
          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/3550#discussion_r107086025

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

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

          +
          +
          + @Test
          + def testAvgSumAggregatationPartition(): Unit = {
          +
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," +
          + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable"
          +
          + val t = StreamTestData.get5TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new ProcTimeTimestamp())
          — End diff –

          @fhueske - I agree but Flink as it operates now does not have the timestamp set to the processing time (basically if you check the timestamp of record it is not set if you do not have this set). Therefore you have 2 possibilities: one: to move the assignment of time in the SQL implementation (as it was initially) or to keep it here and expect this is set.
          before i had this logic in the processing logic - but your remark was that this should not be set there. Therefore now i moved it here.
          I do not see any alternative as this is quite a nasty case where we do not operate just on time nor just on number of elements.

          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/3550#discussion_r107086025 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +317,92 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + + @Test + def testAvgSumAggregatationPartition(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable" + + val t = StreamTestData.get5TupleDataStream(env) + .assignTimestampsAndWatermarks(new ProcTimeTimestamp()) — End diff – @fhueske - I agree but Flink as it operates now does not have the timestamp set to the processing time (basically if you check the timestamp of record it is not set if you do not have this set). Therefore you have 2 possibilities: one: to move the assignment of time in the SQL implementation (as it was initially) or to keep it here and expect this is set. before i had this logic in the processing logic - but your remark was that this should not be set there. Therefore now i moved it here. I do not see any alternative as this is quite a nasty case where we do not operate just on time nor just on number of elements.
          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/3550#discussion_r107086365

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

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testMaxAggregatation(): Unit = {
          — End diff –

          @fhueske i have added those as well. i can keep only those

          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/3550#discussion_r107086365 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +252,179 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testMaxAggregatation(): Unit = { — End diff – @fhueske i have added those as well. i can keep only those
          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/3550#discussion_r107088136

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          @sunjincheng121 @fhueske
          Thanks for posting. I saw the remark also in the other post. I personally do not believe the processingFunction can be used in this case because of several reasons:
          1) if we would use distinct, we cannot extend the implementation - so the design is not extensible/ scalable
          2) Looking at the interface/ example (https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/process_function.html) i do not see how you can do eviction. We need to take into account the fact that for both JIRA (5653 and 5654) the windows are bounded. That means you need to also eliminate - in one based on time in the other based on number of elements. I am not sure how the processing function can support the eviction (do you have any examples for this)?
          3) in case there is not support for eviction that means we need to work with some sort of states in order to keep the elements and manually implement the eviction logic. This would bring a lot of overhead and duplication of code just to not use the windows which directly offer this
          4) we need to consider SQL as a whole (IMHO) - which means that we should consider various other parameters that can be added to the aggregation (for example LIMIT x / TOP y... - see https://issues.apache.org/jira/browse/FLINK-6075?filter=-2). This would require that before you apply the logic you have access to the elements to select the ones that are of interest. With processing function you cannot do this. On the other hand building this logic with windows allows you to extend the implementation to support various SQL features

          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/3550#discussion_r107088136 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( — End diff – @sunjincheng121 @fhueske Thanks for posting. I saw the remark also in the other post. I personally do not believe the processingFunction can be used in this case because of several reasons: 1) if we would use distinct, we cannot extend the implementation - so the design is not extensible/ scalable 2) Looking at the interface/ example ( https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/process_function.html ) i do not see how you can do eviction. We need to take into account the fact that for both JIRA (5653 and 5654) the windows are bounded. That means you need to also eliminate - in one based on time in the other based on number of elements. I am not sure how the processing function can support the eviction (do you have any examples for this)? 3) in case there is not support for eviction that means we need to work with some sort of states in order to keep the elements and manually implement the eviction logic. This would bring a lot of overhead and duplication of code just to not use the windows which directly offer this 4) we need to consider SQL as a whole (IMHO) - which means that we should consider various other parameters that can be added to the aggregation (for example LIMIT x / TOP y... - see https://issues.apache.org/jira/browse/FLINK-6075?filter=-2 ). This would require that before you apply the logic you have access to the elements to select the ones that are of interest. With processing function you cannot do this. On the other hand building this logic with windows allows you to extend the implementation to support various SQL features
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rtudoran commented on the issue:

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

          @fhueske
          "
          I think we cannot use an event-time TimeEvictor to remove records from the GlobalWindow. IMO, we should not automatically overwrite existing timestamps and also not require users to set their own processing-time timestamps. Because of this issue and because a WindowFunction does not support any kind of pre-aggregation, I think that a WindowFunction is not the right approach to implement this type of OVER windows. For example an over window of 1 hour range would aggregate the whole hour for each row over and over again.
          "

          => i am not sure there is any other design/option better than this. We could use:
          -> processing function - see my comments before
          -> we could use window.aggregates (as you suggested to JIRA 5653). There are 2 issues with this: 1) when implementing distinct an the like we still do not have access to the whole data structure ; 2) aggregates is done mainly for supporting just aggregates...which means that in order to support carrying also the contents of the last element (as required by how calcite optimizes the query) you need to artificially create another state-full aggregator to remember the last element (LastElementAggregate) to carry the data - see the solution from @stefanobortoli in 5653 - i feel this is an indication that the design is not the right one either
          ->we could use windows...

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on the issue: https://github.com/apache/flink/pull/3550 @fhueske " I think we cannot use an event-time TimeEvictor to remove records from the GlobalWindow. IMO, we should not automatically overwrite existing timestamps and also not require users to set their own processing-time timestamps. Because of this issue and because a WindowFunction does not support any kind of pre-aggregation, I think that a WindowFunction is not the right approach to implement this type of OVER windows. For example an over window of 1 hour range would aggregate the whole hour for each row over and over again. " => i am not sure there is any other design/option better than this. We could use: -> processing function - see my comments before -> we could use window.aggregates (as you suggested to JIRA 5653). There are 2 issues with this: 1) when implementing distinct an the like we still do not have access to the whole data structure ; 2) aggregates is done mainly for supporting just aggregates...which means that in order to support carrying also the contents of the last element (as required by how calcite optimizes the query) you need to artificially create another state-full aggregator to remember the last element (LastElementAggregate) to carry the data - see the solution from @stefanobortoli in 5653 - i feel this is an indication that the design is not the right one either ->we could use windows...
          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/3550#discussion_r107112009

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time).
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDS.keyBy(partitionKeys:_*)
          + .window(GlobalWindows.create())
          + .trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          — End diff –

          No, it does not work. If you remove the line `.assignTimestampsAndWatermarks(new ProcTimeTimestamp())` from the tests (users should not need to assign timestamps for processing time) the `TimeEvictor` does not evict because `TimeEvictor.hasTimestamp()` returns `false`. `TimeEvictor` is only meant for event time and cannot be used for processing time.

          The reason why the test still work is that no element needs to be evicted because the tests execute too fast.

          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/3550#discussion_r107112009 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDS.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) — End diff – No, it does not work. If you remove the line `.assignTimestampsAndWatermarks(new ProcTimeTimestamp())` from the tests (users should not need to assign timestamps for processing time) the `TimeEvictor` does not evict because `TimeEvictor.hasTimestamp()` returns `false`. `TimeEvictor` is only meant for event time and cannot be used for processing time. The reason why the test still work is that no element needs to be evicted because the tests execute too fast.
          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/3550#discussion_r107112377

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + var i = 0
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          + var reuse: Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          — End diff –

          In Calcite, a LogicalWindow always forwards all fields. Fields which are not needed are removed by a following projection (Calc).

          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/3550#discussion_r107112377 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + var i = 0 + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + var reuse: Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed — End diff – In Calcite, a LogicalWindow always forwards all fields. Fields which are not needed are removed by a following projection (Calc).
          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/3550#discussion_r107113134

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

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

          +
          +
          + @Test
          + def testAvgSumAggregatationPartition(): Unit = {
          +
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," +
          + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable"
          +
          + val t = StreamTestData.get5TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new ProcTimeTimestamp())
          — End diff –

          None of the DataStream operators and none of the Table API operators requires to set a processing timestamp.
          If an operator requires this, its design has flaws and should be reconsidered. We should not use the event-time based `TimeEvictor`.

          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/3550#discussion_r107113134 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +317,92 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + + @Test + def testAvgSumAggregatationPartition(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable" + + val t = StreamTestData.get5TupleDataStream(env) + .assignTimestampsAndWatermarks(new ProcTimeTimestamp()) — End diff – None of the DataStream operators and none of the Table API operators requires to set a processing timestamp. If an operator requires this, its design has flaws and should be reconsidered. We should not use the event-time based `TimeEvictor`.
          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/3550#discussion_r107113328

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

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testMaxAggregatation(): Unit = {
          — End diff –

          Where did you add the tests that check the plan?

          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/3550#discussion_r107113328 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +252,179 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testMaxAggregatation(): Unit = { — End diff – Where did you add the tests that check the plan?
          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/3550#discussion_r107124190

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          HI, @rtudoran thanks for your explanation. I suggested FLINK-5655(event time OVER RANGE BETWEEN x PRECEDING) jointly consider with this JIRA. not the FLINK-5653. In fact, I working on FLINK-5990(event time OVER ROWS BETWEEN x PRECEDING), After open the FLINK-5990's PR, I'll work on the FLINK-5655,then I can share the example.
          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/3550#discussion_r107124190 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( — End diff – HI, @rtudoran thanks for your explanation. I suggested FLINK-5655 (event time OVER RANGE BETWEEN x PRECEDING) jointly consider with this JIRA. not the FLINK-5653 . In fact, I working on FLINK-5990 (event time OVER ROWS BETWEEN x PRECEDING), After open the FLINK-5990 's PR, I'll work on the FLINK-5655 ,then I can share the example. 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/3550#discussion_r107126740

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          @sunjincheng121 thanks a lot for the explanation and i am really curious how eviction is supported on the process function so please ping me when you have the PR.

          Even assuming this can be done very efficient (and not duplicating the code that exists in windows) the other issues still remain: being generic and extendable for the other SQL functions. For us it is important to support the other parameters of SQL as well...distinct, top, limit... and these require a window in the final end...therefore i believe the best design is the one that can be extended and support more functions. I feel it is also in line with the evolution of the flink project that also enabled to be extended for various scenarios.

          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/3550#discussion_r107126740 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( — End diff – @sunjincheng121 thanks a lot for the explanation and i am really curious how eviction is supported on the process function so please ping me when you have the PR. Even assuming this can be done very efficient (and not duplicating the code that exists in windows) the other issues still remain: being generic and extendable for the other SQL functions. For us it is important to support the other parameters of SQL as well...distinct, top, limit... and these require a window in the final end...therefore i believe the best design is the one that can be extended and support more functions. I feel it is also in line with the evolution of the flink project that also enabled to be extended for various scenarios.
          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/3550#discussion_r107126870

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          Actually, I think the `ProcessFunction` is more extensible because the interface is more expressive and allows for more optimizations. We have to manage our own state, but that's rather a good than a bad thing because we have more control over it. That also means that we have to take care of evictions ourselves, i.e., we can have `ListState` and add values or remove them. I agree that it adds more code but not necessarily more overhead. Eviction is not free in DataStream windows, in fact it is rather expensive. Also we would need a custom `Evictor` to remove rows based on processing time.

          As I said before, the `ProcessFunction` allows to continuously accumulate and retract values for aggregations. This will be more efficient than aggregating all rows for each row over and over again. Regarding integration of other SQL operators. SQL operators are assembled in a plan. A `LIMIT` or `TOP` operator will be executed after or before an aggregation but not be merged with the same operator. Moreover, a `ProcessFunction` can execute anything and more than a `WindowFunction`.

          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/3550#discussion_r107126870 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( — End diff – Actually, I think the `ProcessFunction` is more extensible because the interface is more expressive and allows for more optimizations. We have to manage our own state, but that's rather a good than a bad thing because we have more control over it. That also means that we have to take care of evictions ourselves, i.e., we can have `ListState` and add values or remove them. I agree that it adds more code but not necessarily more overhead. Eviction is not free in DataStream windows, in fact it is rather expensive. Also we would need a custom `Evictor` to remove rows based on processing time. As I said before, the `ProcessFunction` allows to continuously accumulate and retract values for aggregations. This will be more efficient than aggregating all rows for each row over and over again. Regarding integration of other SQL operators. SQL operators are assembled in a plan. A `LIMIT` or `TOP` operator will be executed after or before an aggregation but not be merged with the same operator. Moreover, a `ProcessFunction` can execute anything and more than a `WindowFunction`.
          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/3550#discussion_r107126965

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

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

          +
          +
          + @Test
          + def testAvgSumAggregatationPartition(): Unit = {
          +
          + val env = StreamExecutionEnvironment.getExecutionEnvironment
          + val tEnv = TableEnvironment.getTableEnvironment(env)
          + env.setParallelism(1)
          + StreamITCase.testResults = mutable.MutableList()
          +
          + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," +
          + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" +
          + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable"
          +
          + val t = StreamTestData.get5TupleDataStream(env)
          + .assignTimestampsAndWatermarks(new ProcTimeTimestamp())
          — End diff –

          @fhueske - what other option is there to support time boundaries if not time evictor?

          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/3550#discussion_r107126965 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala — @@ -317,4 +317,92 @@ class SqlITCase extends StreamingWithStateTestBase { result.addSink(new StreamITCase.StringSink) env.execute() } + + + @Test + def testAvgSumAggregatationPartition(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + env.setParallelism(1) + StreamITCase.testResults = mutable.MutableList() + + val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS avgC," + + "SUM(c) OVER (PARTITION BY a ORDER BY procTime()" + + "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) as sumC FROM MyTable" + + val t = StreamTestData.get5TupleDataStream(env) + .assignTimestampsAndWatermarks(new ProcTimeTimestamp()) — End diff – @fhueske - what other option is there to support time boundaries if not time evictor?
          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/3550#discussion_r107127833

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala —
          @@ -119,6 +152,60 @@ class DataStreamOverAggregate(

          }

          + def createTimeBoundedProcessingTimeOverWindow(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
          +
          + val index = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
          + val count = input.getRowType().getFieldCount()
          + val lowerboundIndex = index - count
          +
          +
          + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match

          { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + }

          +
          + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions(
          + namedAggregates.map(_.getKey),inputType, needRetraction = false)
          +
          +
          + // As we it is not possible to operate neither on sliding count neither
          + // on sliding time we need to manage the eviction of the events that
          + // expire ourselves based on the proctime (system time).
          +
          + // get the output types
          + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
          +
          + val result: DataStream[Row] =
          + if (partitionKeys.nonEmpty) {
          + inputDS.keyBy(partitionKeys:_*)
          + .window(GlobalWindows.create())
          + .trigger(CountTrigger.of(1))
          + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
          — End diff –

          @fhueske i tried also with evicting events and it does work if the .assignTimestampsAndWatermarks is set.
          from my point of view if the environment has the ProcessingTime set ...than from my point of view the timestamp should be automatically set to the processing time.
          What is your suggestion to handle this issue...we still need to evict based on processing time.

          My initial design was to hide this and have this function within the implementation. With that the users do not need to set it and it does not mess up anything ...because in the final end the processing that we build and for which we setup the timestamp is only specific to this query that has the processing time as marker to operate on it.
          I would suggest to put it back as it was - what do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3550#discussion_r107127833 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala — @@ -119,6 +152,60 @@ class DataStreamOverAggregate( } + def createTimeBoundedProcessingTimeOverWindow(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 + + val index = overWindow.lowerBound.getOffset.asInstanceOf [RexInputRef] .getIndex + val count = input.getRowType().getFieldCount() + val lowerboundIndex = index - count + + + val time_boundary = logicWindow.constants.get(lowerboundIndex).getValue2 match { + case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex) + .getValue2.asInstanceOf[java.math.BigDecimal].longValue() + case _ => throw new TableException("OVER Window boundaries must be numeric") + } + + val (aggFields, aggregates) = AggregateUtil.transformToAggregateFunctions( + namedAggregates.map(_.getKey),inputType, needRetraction = false) + + + // As we it is not possible to operate neither on sliding count neither + // on sliding time we need to manage the eviction of the events that + // expire ourselves based on the proctime (system time). + + // get the output types + val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf [RowTypeInfo] + + val result: DataStream [Row] = + if (partitionKeys.nonEmpty) { + inputDS.keyBy(partitionKeys:_*) + .window(GlobalWindows.create()) + .trigger(CountTrigger.of(1)) + .evictor(TimeEvictor.of(Time.milliseconds(time_boundary))) — End diff – @fhueske i tried also with evicting events and it does work if the .assignTimestampsAndWatermarks is set. from my point of view if the environment has the ProcessingTime set ...than from my point of view the timestamp should be automatically set to the processing time. What is your suggestion to handle this issue...we still need to evict based on processing time. My initial design was to hide this and have this function within the implementation. With that the users do not need to set it and it does not mess up anything ...because in the final end the processing that we build and for which we setup the timestamp is only specific to this query that has the processing time as marker to operate on it. I would suggest to put it back as it was - what do you think?
          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/3550#discussion_r107131154

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

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testMaxAggregatation(): Unit = {
          — End diff –

          I thought you refer to what i added in scala/stream/sql/SqlITCase.scala
          ...which is the partitioned and non-partitioned verification of 2 aggregates

          ..i am not sure what you mean with checking the plan....
          @fhueske

          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/3550#discussion_r107131154 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +252,179 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testMaxAggregatation(): Unit = { — End diff – I thought you refer to what i added in scala/stream/sql/SqlITCase.scala ...which is the partitioned and non-partitioned verification of 2 aggregates ..i am not sure what you mean with checking the plan.... @fhueske
          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/3550#discussion_r107133004

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          @fhueske

          I will start with the second part - supporting LIMIT, Distinct, top.... even if these leads to a separate operator that would be applied in theory before we should optimize and combine them...it does not make sense to have 2 operators for one logical operation. Secondly if we refer to keeping them separately - the logic would be that you have a window where you keep all the data to apply limit/top/... and than you fire multiple elements that you selected to the operator to compute the aggregator - so you would reset the aggregator every time.

          Regarding your remark of managing the state in process function by ourselves...we are basically creating a window if we keep all elements in the state of the process function. we still need to go through the potentially all elements to evict them and update the aggregator. I am not sure which is more efficiently managed: a large function state or the buffered elements of the window. I would assume in the best case they are similar in performance... if window buffers are not optimized. Nevertheless...there is no advantage as far as i see and understand the problem of having the process function ...but i am sure it brings extra overhead and limitations in the future.

          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/3550#discussion_r107133004 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( — End diff – @fhueske I will start with the second part - supporting LIMIT, Distinct, top.... even if these leads to a separate operator that would be applied in theory before we should optimize and combine them...it does not make sense to have 2 operators for one logical operation. Secondly if we refer to keeping them separately - the logic would be that you have a window where you keep all the data to apply limit/top/... and than you fire multiple elements that you selected to the operator to compute the aggregator - so you would reset the aggregator every time. Regarding your remark of managing the state in process function by ourselves...we are basically creating a window if we keep all elements in the state of the process function. we still need to go through the potentially all elements to evict them and update the aggregator. I am not sure which is more efficiently managed: a large function state or the buffered elements of the window. I would assume in the best case they are similar in performance... if window buffers are not optimized. Nevertheless...there is no advantage as far as i see and understand the problem of having the process function ...but i am sure it brings extra overhead and limitations in the future.
          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/3550#discussion_r107133216

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          + private val aggregates: Array[AggregateFunction[_]],
          + private val aggFields: Array[Int],
          + private val forwardedFieldCount: Int)
          + extends RichAllWindowFunction[Row, Row, W] {
          +
          + private var output: Row = _
          + private var accumulators: Row = _
          +
          + override def open(parameters: Configuration): Unit = {
          + output = new Row(forwardedFieldCount + aggregates.length)
          + accumulators = new Row(aggregates.length)
          + var i = 0
          + while (i < aggregates.length)

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

          + }
          +
          + /**
          + * Calculate aggregated values output by aggregate buffer, and set them into output
          + * Row based on the mapping relation between intermediate aggregate data and output data.
          + */
          + override def apply(
          + window: W,
          + records: Iterable[Row],
          + out: Collector[Row]): Unit = {
          +
          +
          + //initialize the values of the aggregators by re-creating them
          + //the design of the Accumulator interface should be extended to enable
          + //a reset function for better performance
          + var i = 0
          + while (i < aggregates.length)

          { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + }

          + var reuse: Row = null
          + //iterate through the elements and aggregate
          + val iter = records.iterator
          + while (iter.hasNext) {
          + reuse = iter.next
          + i = 0
          + while (i < aggregates.length)

          { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + }

          + }
          +
          + //set the values of the result with current elements values if needed
          — End diff –

          Cool - thanks for the explanation i will update the comment

          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/3550#discussion_r107133216 — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala — @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.functions.Accumulator + +import java.lang.Iterable +import org.apache.flink.table.functions.AggregateFunction + +/** + * Computes the final aggregate value from incrementally computed aggreagtes. + * + * @param aggregates The aggregates to be computed + * @param aggFields the fields on which to apply the aggregate. + * @param forwardedFieldCount The fields to be carried from current row. + */ +class DataStreamProcTimeAggregateGlobalWindowFunction [W <: Window] ( + private val aggregates: Array[AggregateFunction [_] ], + private val aggFields: Array [Int] , + private val forwardedFieldCount: Int) + extends RichAllWindowFunction [Row, Row, W] { + + private var output: Row = _ + private var accumulators: Row = _ + + override def open(parameters: Configuration): Unit = { + output = new Row(forwardedFieldCount + aggregates.length) + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i = i + 1 + } + } + + /** + * Calculate aggregated values output by aggregate buffer, and set them into output + * Row based on the mapping relation between intermediate aggregate data and output data. + */ + override def apply( + window: W, + records: Iterable [Row] , + out: Collector [Row] ): Unit = { + + + //initialize the values of the aggregators by re-creating them + //the design of the Accumulator interface should be extended to enable + //a reset function for better performance + var i = 0 + while (i < aggregates.length) { + aggregates(i).resetAccumulator(accumulators.getField(i).asInstanceOf[Accumulator]) + i += 1 + } + var reuse: Row = null + //iterate through the elements and aggregate + val iter = records.iterator + while (iter.hasNext) { + reuse = iter.next + i = 0 + while (i < aggregates.length) { + val accumulator = accumulators.getField(i).asInstanceOf[Accumulator] + aggregates(i).accumulate(accumulator, reuse.getField(aggFields(i))) + i += 1 + } + } + + //set the values of the result with current elements values if needed — End diff – Cool - thanks for the explanation i will update the comment
          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/3550#discussion_r107155069

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

          { ) streamUtil.verifySql(sql, expected) }

          +
          + @Test
          + def testMaxAggregatation(): Unit = {
          — End diff –

          Basically implement tests which are equivalent to the other tests in `WindowAggregateTest`. These tests do not execute a query but compare the plans generated by the optimizer.

          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/3550#discussion_r107155069 — Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala — @@ -239,4 +252,179 @@ class WindowAggregateTest extends TableTestBase { ) streamUtil.verifySql(sql, expected) } + + @Test + def testMaxAggregatation(): Unit = { — End diff – Basically implement tests which are equivalent to the other tests in `WindowAggregateTest`. These tests do not execute a query but compare the plans generated by the optimizer.
          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/3550#discussion_r107157242

          — Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataStreamProcTimeAggregateGlobalWindowFunction.scala —
          @@ -0,0 +1,106 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.streaming.api.functions.windowing.RichAllWindowFunction
          +import org.apache.flink.types.Row
          +import org.apache.flink.util.Collector
          +import org.apache.flink.streaming.api.windowing.windows.Window
          +import org.apache.flink.configuration.Configuration
          +import org.apache.flink.table.functions.Accumulator
          +
          +import java.lang.Iterable
          +import org.apache.flink.table.functions.AggregateFunction
          +
          +/**
          + * Computes the final aggregate value from incrementally computed aggreagtes.
          + *
          + * @param aggregates The aggregates to be computed
          + * @param aggFields the fields on which to apply the aggregate.
          + * @param forwardedFieldCount The fields to be carried from current row.
          + */
          +class DataStreamProcTimeAggregateGlobalWindowFunction[W <: Window](
          — End diff –

          A `ProcessFunction` would look like this:

          • `open()` In open we create two types of state: 1) the accumulators, which is a `ValueState[Row]`, 2) the rows we need to retract and their processing timestamp. This can be a `ListState[Tuple2[Long, Row]]`.
          • `processElement()` is called for each arriving row. We accumulate the row to the accumulator (fetched from the state) and retract all value from the list which fall out of the range (could also be none). We get the current processing time from the Context object. Since the list is sorted on insertion order, we take the head elements. We emit the row with the updated accumulator and append it to the list of rows to retract together with its processing timestamp (again, current time taken from the Context object). Finally we update both states.
          • `onTimer()` is not required for processing time OVER window.

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

          What do you think?