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

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

Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/2938
  
    Here is an example to reproduce the wrong results:
    
    ```scala
      @Test
      def testEventTimeTumblingWindowStream(): Unit = {
        val env = StreamExecutionEnvironment.getExecutionEnvironment
        env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
        val tEnv = TableEnvironment.getTableEnvironment(env)
    
        val stream = env
          .fromCollection(data)
          .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
        val table = stream.toTable(tEnv, 'long, 'int, 'string)
    
        val windowedTable = table
          .groupBy('string)
          .window(Tumble over 5.milli on 'rowtime as 'w)
          .select('string, 'int.count, 'int.avg, 'w.start, 'w.end)
    
        val results = windowedTable.toDataStream[Row].print()
        env.execute()
      }
    
      @Test
      def testEventTimeTumblingWindowBatch(): Unit = {
        val env = ExecutionEnvironment.getExecutionEnvironment
        val tEnv = TableEnvironment.getTableEnvironment(env)
    
        val stream = env
          .fromCollection(data)
        val table = stream.toTable(tEnv, 'long, 'int, 'string)
    
        val windowedTable = table
          .groupBy('string)
          .window(Tumble over 5.milli on 'long as 'w)
          .select('string, 'int.count, 'int.avg, 'w.start, 'w.end)
    
        val results = windowedTable.toDataSet[Row].print()
      }
    ```
    
    It should be: 
    ```
    Hello world,1,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01
    Hello,2,2,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005
    Hello world,1,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02
    Hi,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005
    ```
    
    But is:
    ```
    Hello,2,2,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005
    Hello world,2,3,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005
    Hi,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005
    ```


> Add tumbling group-windows for batch tables
> -------------------------------------------
>
>                 Key: FLINK-4692
>                 URL: https://issues.apache.org/jira/browse/FLINK-4692
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Timo Walther
>            Assignee: Jark Wu
>
> Add Tumble group-windows for batch tables as described in 
> [FLIP-11|https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations].
>  



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to