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?
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---