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

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

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

    https://github.com/apache/flink/pull/3733#discussion_r113043078
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
 ---
    @@ -627,24 +640,27 @@ abstract class TableEnvironment(val config: 
TableConfig) {
         * @param functionName name of the map function. Must not be unique but 
has to be a
         *                     valid Java class identifier.
         */
    -  protected def sinkConversion[T](
    -      physicalRowTypeInfo: TypeInformation[Row],
    +  protected def sinkConversion[T, P](
    --- End diff --
    
    I would refactor this as follows:
    - make `sinkConversion()` an abstract method that is implemented by 
`BatchTableEnvironemnt` and `StreamTableEnvironment`
    - extract the code-generation part that converts a `Row` into the requested 
data type and keep this in a method 
`TableEnvironment.generateRowConverterFunction()`
    - the implementations of `sinkConversion()` can use the 
`generateRowConverterFunction`


> Implement and turn on the retraction for aggregates
> ---------------------------------------------------
>
>                 Key: FLINK-6091
>                 URL: https://issues.apache.org/jira/browse/FLINK-6091
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Shaoxuan Wang
>            Assignee: Hequn Cheng
>
> Implement functions for generating and consuming retract messages for 
> different aggregates. 
> 1. add delete/add property to Row
> 2. implement functions for generating retract messages for unbounded groupBy
> 3. implement functions for handling retract messages for different aggregates.
> 4. handle retraction messages in CommonCorrelate and CommonCalc (retain 
> Delete property).
> Note: Currently, only unbounded groupby generates retraction and it is 
> working under unbounded and processing time mode. Hence, retraction is only 
> supported for unbounded and processing time aggregations so far. We can add 
> more retraction support later.
> supported now: unbounded groupby, unbounded and processing time over window
> unsupported now: group window, event time or bounded over window.



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

Reply via email to