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

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

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

    https://github.com/apache/flink/pull/3386#discussion_r102922045
  
    --- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
 ---
    @@ -171,4 +175,98 @@ class SqlITCase extends 
StreamingMultipleProgramsTestBase {
         val expected = mutable.MutableList("Hello", "Hello world")
         assertEquals(expected.sorted, StreamITCase.testResults.sorted)
       }
    +
    +  /** test sliding event-time unbounded window with partition by **/
    +  @Test
    +  def testSlideEventTimeUnboundWindowWithGroup(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val sqlQuery = "SELECT a, b, SUM(a) over (partition by b order by 
rowtime() range between " +
    +      "unbounded preceding and current row) from T1"
    +
    +    val t1 = StreamTestData.getSmall3TupleDataStream(env)
    +      .assignTimestampsAndWatermarks(new 
AssignerWithPeriodicWatermarks[(Int, Long, String)] {
    +
    +        def getCurrentWatermark: Watermark = new 
Watermark(System.currentTimeMillis)
    +
    +        def extractTimestamp(element: (Int, Long, String), 
previousElementTimestamp: Long): Long =
    +          1400000
    +      }).toTable(tEnv).as('a, 'b, 'c)
    +    tEnv.registerTable("T1", t1)
    +
    +    val result = tEnv.sql(sqlQuery).toDataStream[Row]
    +    result.addSink(new StreamITCase.StringSink)
    +    env.execute()
    +
    +    val expected1 = mutable.MutableList(
    +      "1,1,1", "2,2,2", "3,2,5")
    +    val expected2 = mutable.MutableList(
    +      "1,1,1", "2,2,5", "3,2,3")
    +    assertTrue(expected1.equals(StreamITCase.testResults.sorted) ||
    +      expected2.equals(StreamITCase.testResults.sorted))
    +  }
    +
    +  /** test sliding event-time unbounded window without partitiion by **/
    +  @Test
    +  def testSlideEventTimeUnboundWindowWithoutGroup(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val sqlQuery = "SELECT SUM(a) " +
    +      "over (order by rowtime() range between unbounded preceding and 
current row) from T1"
    +
    +    val t1 = StreamTestData.getSmall3TupleDataStream(env)
    +      .assignTimestampsAndWatermarks(new 
AssignerWithPeriodicWatermarks[(Int, Long, String)] {
    +
    +        def getCurrentWatermark: Watermark = new 
Watermark(System.currentTimeMillis)
    +
    +        def extractTimestamp(element: (Int, Long, String), 
previousElementTimestamp: Long): Long =
    +          1400000
    +      }).toTable(tEnv).as('a, 'b, 'c)
    +    tEnv.registerTable("T1", t1)
    +
    +    val result = tEnv.sql(sqlQuery).toDataStream[Row]
    +    result.addSink(new StreamITCase.StringSink)
    +    env.execute()
    +
    +    assertEquals(Some("6"), StreamITCase.testResults.sorted.get(2))
    +  }
    +
    +  /** test sliding event-time unbounded window with later record **/
    +  @Test
    +  def testSlideEventTimeUnboundWindowWithLater(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    env.setParallelism(1)
    --- End diff --
    
    When you remove this configuration, run many times will get different 
results, such as:
    I have added a time column for the convenience of observation `SUM(a) -> d, 
SUM(a) `:
    
    The first time:
    ```
    1400005,2
    1400010,6
    ```
    The second time
    ```
    1400000,1
    1400010,5
    ```
    The third time
    ```
    1300000,3
    1400005,5
    1400010,9
    ```
    Do you have that happen?If so, please check and fix the bug?


> Add event time OVER RANGE BETWEEN UNBOUNDED PRECEDING aggregation to SQL
> ------------------------------------------------------------------------
>
>                 Key: FLINK-5658
>                 URL: https://issues.apache.org/jira/browse/FLINK-5658
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: Yuhong Hong
>
> The goal of this issue is to add support for OVER RANGE aggregations on event 
> 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 rowTime() RANGE BETWEEN UNBOUNDED 
> PRECEDING AND CURRENT ROW) AS sumB,
>   MIN(b) OVER (PARTITION BY c ORDER BY rowTime() RANGE BETWEEN UNBOUNDED 
> 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 rowTime() as parameter. rowTime() is a 
> parameterless scalar function that just indicates processing time mode.
> - bounded PRECEDING is not supported (see FLINK-5655)
> - 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)

Reply via email to