[
https://issues.apache.org/jira/browse/FLINK-4691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15548969#comment-15548969
]
ASF GitHub Bot commented on FLINK-4691:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/2562#discussion_r81860061
--- Diff:
flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/GroupWindowITCase.scala
---
@@ -0,0 +1,777 @@
+/*
+ * 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.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import
org.apache.flink.api.scala.stream.table.GroupWindowITCase.TimestampWithEqualWatermark
+import org.apache.flink.api.scala.stream.utils.StreamITCase
+import org.apache.flink.api.scala.table._
+import org.apache.flink.api.table.{Row, _}
+import org.apache.flink.streaming.api.TimeCharacteristic
+import
org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.watermark.Watermark
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+class GroupWindowITCase extends StreamingMultipleProgramsTestBase {
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidBatchWindow(): Unit = {
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Session withGap 10.rows as 'string)
+ }
+
+ @Test(expected = classOf[TableException])
+ def testInvalidRowtime1(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'rowtime, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Tumble over 50.milli)
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidRowtime2(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Tumble over 50.milli)
+ .select('string, 'int.count as 'rowtime)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidRowtime3(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table.as('rowtime, 'myint, 'mystring)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidRowtime4(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Tumble over 50.milli on 'string)
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidTumblingSize(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Tumble over "WRONG")
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidSlidingSize(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Slide over "WRONG" every "WRONG")
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidSlidingSlide(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Slide over 12.rows every "WRONG")
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidSessionGap(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Session withGap 10.rows)
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidWindowAlias1(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Session withGap 10.rows as 1 + 1)
+ .select('string, 'int.count)
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidWindowAlias2(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val data = new mutable.MutableList[(Long, Int, String)]
+ val stream = env.fromCollection(data)
+ val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+ table
+ .groupBy('string)
+ .window(Session withGap 10.rows as 'string)
--- End diff --
is this test failing for the right reason? `Session withGap 10.rows` is
another possible cause.
> Add group-windows for streaming tables
> ---------------------------------------
>
> Key: FLINK-4691
> URL: https://issues.apache.org/jira/browse/FLINK-4691
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: Timo Walther
> Assignee: Timo Walther
>
> Add Tumble, Slide, Session group-windows for streaming tables as described in
> [FLIP-11|https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations].
>
> Implementation of group-windows on streaming tables. This includes
> implementing the API of group-windows, the logical validation for
> group-windows, and the definition of the “rowtime” and “systemtime” keywords.
> Group-windows on batch tables won’t be initially supported and will throw an
> exception.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)