You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2017/06/19 01:57:50 UTC

flink git commit: [FLINK-6930] [table] Forbid selecting window start/end on row-based Tumble/Slide windows

Repository: flink
Updated Branches:
  refs/heads/master afb0b630c -> 06e63386e


[FLINK-6930] [table] Forbid selecting window start/end on row-based Tumble/Slide windows

This closes #4133


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/06e63386
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/06e63386
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/06e63386

Branch: refs/heads/master
Commit: 06e63386e56e0bf73030b1bf31890c8103592fc0
Parents: afb0b63
Author: Jark Wu <wu...@alibaba-inc.com>
Authored: Fri Jun 16 11:49:31 2017 +0800
Committer: Jark Wu <wu...@alibaba-inc.com>
Committed: Mon Jun 19 09:55:32 2017 +0800

----------------------------------------------------------------------
 .../flink/table/plan/logical/operators.scala    | 16 ++++++++++++++
 .../scala/stream/table/GroupWindowTest.scala    | 22 ++++++++++++++++++++
 2 files changed, 38 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/06e63386/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
index 6777ef5..4077d36 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
@@ -31,6 +31,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.operators.join.JoinType
 import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment, UnresolvedException}
 import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
+import org.apache.flink.table.expressions.ExpressionUtils.isRowCountLiteral
 import org.apache.flink.table.expressions._
 import org.apache.flink.table.functions.TableFunction
 import org.apache.flink.table.functions.utils.TableSqlFunction
@@ -652,6 +653,21 @@ case class WindowAggregate(
       case ValidationSuccess => // ok
     }
 
+    // validate property
+    if (propertyExpressions.nonEmpty) {
+      resolvedWindowAggregate.window match {
+        case TumblingGroupWindow(_, _, size) if isRowCountLiteral(size) =>
+          failValidation("Window start and Window end cannot be selected " +
+                           "for a row-count Tumbling window.")
+
+        case SlidingGroupWindow(_, _, size, _) if isRowCountLiteral(size) =>
+          failValidation("Window start and Window end cannot be selected " +
+                           "for a row-count Sliding window.")
+
+        case _ => // ok
+      }
+    }
+
     resolvedWindowAggregate
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/06e63386/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
index 55689d0..593b036 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
@@ -185,6 +185,28 @@ class GroupWindowTest extends TableTestBase {
       .select('string, weightedAvg('string, 'int)) // invalid UDAGG args
   }
 
+  @Test(expected = classOf[ValidationException])
+  def testInvalidWindowPropertyOnRowCountsTumblingWindow(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
+
+    table
+    .window(Tumble over 2.rows on 'proctime as 'w)
+    .groupBy('w, 'string)
+    .select('string, 'w.start, 'w.end) // invalid start/end on rows-count window
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidWindowPropertyOnRowCountsSlidingWindow(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
+
+    table
+    .window(Slide over 10.rows every 5.rows on 'proctime as 'w)
+    .groupBy('w, 'string)
+    .select('string, 'w.start, 'w.end) // invalid start/end on rows-count window
+  }
+
   @Test
   def testMultiWindow(): Unit = {
     val util = streamTestUtil()