flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-5654) Add processing time OVER RANGE BETWEEN x PRECEDING aggregation to SQL
Date Fri, 17 Mar 2017 11:35:42 GMT

    [ https://issues.apache.org/jira/browse/FLINK-5654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15929791#comment-15929791
] 

ASF GitHub Bot commented on FLINK-5654:
---------------------------------------

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(i).createAccumulator())
    --- End diff --
    
    Actually, accumulators can be created once in `open()` and be reused after the have been
reset with `AggregateFunction.resetAccumulator()`


> Add processing time OVER RANGE BETWEEN x PRECEDING aggregation to SQL
> ---------------------------------------------------------------------
>
>                 Key: FLINK-5654
>                 URL: https://issues.apache.org/jira/browse/FLINK-5654
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: radu
>
> 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:
> {code}
> 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
> {code}
> 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).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message