You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2017/05/05 23:52:35 UTC

[01/15] flink git commit: [FLINK-5884] [table] Integrate time indicators for Table API & SQL

Repository: flink
Updated Branches:
  refs/heads/master 28ab73750 -> f37988c19


http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/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 084ee14..de6cbfa 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
@@ -20,9 +20,9 @@ package org.apache.flink.table.api.scala.stream.table
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.java.utils.UserDefinedAggFunctions.WeightedAvgWithMerge
+import org.apache.flink.table.api.ValidationException
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{TableException, ValidationException}
-import org.apache.flink.table.expressions.{RowtimeAttribute, WindowReference}
+import org.apache.flink.table.expressions.WindowReference
 import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.utils.TableTestBase
 import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode}
@@ -40,46 +40,10 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'string.start) // property in non windowed table
   }
 
-  @Test(expected = classOf[TableException])
-  def testInvalidRowtime1(): Unit = {
-    val util = streamTestUtil()
-    // rowtime attribute must not be a field name
-    util.addTable[(Long, Int, String)]('rowtime, 'long, 'int, 'string)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidRowtime2(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .select('string, 'int as 'rowtime) // rowtime attribute must not be an alias
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidRowtime3(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table.as('rowtime, 'myint, 'mystring) // rowtime attribute must not be an alias
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidRowtime4(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-    // only rowtime is a valid time attribute in a stream environment
-      .window(Tumble over 50.milli on 'string as 'w)
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
-  }
-
   @Test(expected = classOf[ValidationException])
   def testGroupByWithoutWindowAlias(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
       .window(Tumble over 5.milli on 'long as 'w)
@@ -90,7 +54,7 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidRowTimeRef(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
       .window(Tumble over 5.milli on 'long as 'w)
@@ -104,10 +68,10 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidTumblingSize(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
-      .window(Tumble over "WRONG" as 'w) // string is not a valid interval
+      .window(Tumble over "WRONG" on 'long as 'w) // string is not a valid interval
       .groupBy('w, 'string)
       .select('string, 'int.count)
   }
@@ -127,10 +91,10 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidSlidingSize(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
-      .window(Slide over "WRONG" every "WRONG" as 'w) // string is not a valid interval
+      .window(Slide over "WRONG" every "WRONG" on 'long as 'w) // string is not a valid interval
       .groupBy('w, 'string)
       .select('string, 'int.count)
   }
@@ -138,10 +102,11 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidSlidingSlide(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
-      .window(Slide over 12.rows every 1.minute as 'w) // row and time intervals may not be mixed
+      // row and time intervals may not be mixed
+      .window(Slide over 12.rows every 1.minute on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
   }
@@ -161,10 +126,11 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidSessionGap(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
-      .window(Session withGap 10.rows as 'w) // row interval is not valid for session windows
+      // row interval is not valid for session windows
+      .window(Session withGap 10.rows on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
   }
@@ -172,10 +138,10 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidWindowAlias1(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
-      .window(Session withGap 100.milli as 1 + 1) // expression instead of a symbol
+      .window(Session withGap 100.milli on 'long as 1 + 1) // expression instead of a symbol
       .groupBy('string)
       .select('string, 'int.count)
   }
@@ -183,10 +149,11 @@ class GroupWindowTest extends TableTestBase {
   @Test(expected = classOf[ValidationException])
   def testInvalidWindowAlias2(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     table
-      .window(Session withGap 100.milli as 'string) // field name "string" is already present
+      // field name "string" is already present
+      .window(Session withGap 100.milli on 'long as 'string)
       .groupBy('string)
       .select('string, 'int.count)
   }
@@ -195,7 +162,7 @@ class GroupWindowTest extends TableTestBase {
   def testSessionUdAggWithInvalidArgs(): Unit = {
     val util = streamTestUtil()
     val weightedAvg = new WeightedAvgWithMerge
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
 
     table
       .window(Session withGap 2.hours on 'rowtime as 'w)
@@ -203,16 +170,17 @@ class GroupWindowTest extends TableTestBase {
       .select('string, weightedAvg('string, 'int)) // invalid UDAGG args
   }
 
+  @Ignore // TODO
   @Test
   def testMultiWindow(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Tumble over 50.milli as 'w1)
+      .window(Tumble over 50.milli on 'proctime as 'w1)
       .groupBy('w1, 'string)
-      .select('string, 'int.count)
-      .window(Slide over 20.milli every 10.milli as 'w2)
+      .select('w.end as 'proctime, 'string, 'int.count)
+      .window(Slide over 20.milli every 10.milli on 'proctime as 'w2)
       .groupBy('w2)
       .select('string.count)
 
@@ -230,8 +198,9 @@ class GroupWindowTest extends TableTestBase {
           term("groupBy", "string"),
           term(
             "window",
-            ProcessingTimeTumblingGroupWindow(
+            TumblingGroupWindow(
               WindowReference("w1"),
+              'proctime,
               50.milli)),
           term("select", "string", "COUNT(int) AS TMP_0")
         ),
@@ -239,9 +208,11 @@ class GroupWindowTest extends TableTestBase {
       ),
       term(
         "window",
-        ProcessingTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w2"),
-          20.milli, 10.milli)),
+          'proctime,
+          20.milli,
+          10.milli)),
       term("select", "COUNT(string) AS TMP_1")
     )
     util.verifyTable(windowedTable, expected)
@@ -250,10 +221,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testProcessingTimeTumblingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Tumble over 50.milli as 'w)
+      .window(Tumble over 50.milli on 'proctime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -262,13 +233,14 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "string", "int")
+        term("select", "string", "int", "proctime")
       ),
       term("groupBy", "string"),
       term(
         "window",
-        ProcessingTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
+          'proctime,
           50.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
@@ -279,38 +251,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testProcessingTimeTumblingGroupWindowOverCount(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Tumble over 2.rows as 'w)
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term(
-        "window",
-        ProcessingTimeTumblingGroupWindow(
-          WindowReference("w"), 2.rows)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testEventTimeTumblingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Tumble over 5.milli on 'rowtime as 'w)
+      .window(Tumble over 2.rows on 'proctime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -319,15 +263,15 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "string", "int")
+        term("select", "string", "int", "proctime")
       ),
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(),
-          5.milli)),
+          'proctime,
+          2.rows)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -335,13 +279,12 @@ class GroupWindowTest extends TableTestBase {
   }
 
   @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testEventTimeTumblingGroupWindowOverCount(): Unit = {
+  def testEventTimeTumblingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Tumble over 2.rows on 'rowtime as 'w)
+      .window(Tumble over 5.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -351,9 +294,10 @@ class GroupWindowTest extends TableTestBase {
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(), 2.rows)),
+          'long,
+          5.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -363,7 +307,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeTumblingGroupWindowWithUdAgg(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
 
     val weightedAvg = new WeightedAvgWithMerge
 
@@ -378,9 +322,9 @@ class GroupWindowTest extends TableTestBase {
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(),
+          'rowtime,
           5.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
@@ -391,10 +335,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testProcessingTimeSlidingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Slide over 50.milli every 50.milli as 'w)
+      .window(Slide over 50.milli every 50.milli on 'proctime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -403,14 +347,16 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "string", "int")
+        term("select", "string", "int", "proctime")
       ),
       term("groupBy", "string"),
       term(
         "window",
-        ProcessingTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          50.milli, 50.milli)),
+          'proctime,
+          50.milli,
+          50.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -420,10 +366,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Slide over 2.rows every 1.rows as 'w)
+      .window(Slide over 2.rows every 1.rows on 'proctime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -432,14 +378,16 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "string", "int")
+        term("select", "string", "int", "proctime")
       ),
       term("groupBy", "string"),
       term(
         "window",
-        ProcessingTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          2.rows, 1.rows)),
+          'proctime,
+          2.rows,
+          1.rows)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -449,40 +397,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeSlidingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Slide over 8.milli every 10.milli on 'rowtime as 'w)
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term(
-        "window",
-        EventTimeSlidingGroupWindow(
-          WindowReference("w"),
-          RowtimeAttribute(), 8.milli, 10.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testEventTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Slide over 2.rows every 1.rows on 'rowtime as 'w)
+      .window(Slide over 8.milli every 10.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -492,9 +410,11 @@ class GroupWindowTest extends TableTestBase {
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(), 2.rows, 1.rows)),
+          'long,
+          8.milli,
+          10.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -504,7 +424,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeSlidingGroupWindowWithUdAgg(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
 
     val weightedAvg = new WeightedAvgWithMerge
 
@@ -519,9 +439,7 @@ class GroupWindowTest extends TableTestBase {
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeSlidingGroupWindow(
-          WindowReference("w"),
-          RowtimeAttribute(), 8.milli, 10.milli)),
+        SlidingGroupWindow(WindowReference("w"), 'rowtime, 8.milli, 10.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -531,26 +449,23 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeSessionGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Session withGap 7.milli on 'rowtime as 'w)
+      .window(Session withGap 7.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
     val expected = unaryNode(
       "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
+      streamTableNode(0),
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeSessionGroupWindow(
+        SessionGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(), 7.milli)),
+          'long,
+          7.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -560,7 +475,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeSessionGroupWindowWithUdAgg(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
 
     val weightedAvg = new WeightedAvgWithMerge
 
@@ -575,9 +490,7 @@ class GroupWindowTest extends TableTestBase {
       term("groupBy", "string"),
       term(
         "window",
-        EventTimeSessionGroupWindow(
-          WindowReference("w"),
-          RowtimeAttribute(), 7.milli)),
+        SessionGroupWindow(WindowReference("w"), 'rowtime, 7.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -587,10 +500,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllProcessingTimeTumblingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Tumble over 50.milli as 'w)
+      .window(Tumble over 50.milli on 'proctime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count)
 
@@ -599,13 +512,14 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "string", "int")
+        term("select", "string", "int", "proctime")
       ),
       term("groupBy", "string"),
       term(
         "window",
-        ProcessingTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
+          'proctime,
           50.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
@@ -616,10 +530,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Tumble over 2.rows as 'w)
+      .window(Tumble over 2.rows on 'proctime as 'w)
       .groupBy('w)
       .select('int.count)
 
@@ -628,12 +542,13 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "int")
+        term("select", "int", "proctime")
       ),
       term(
         "window",
-        ProcessingTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
+          'proctime,
           2.rows)),
       term("select", "COUNT(int) AS TMP_0")
     )
@@ -644,39 +559,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllEventTimeTumblingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Tumble over 5.milli on 'rowtime as 'w)
-      .groupBy('w)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term(
-        "window",
-        EventTimeTumblingGroupWindow(
-          WindowReference("w"),
-          RowtimeAttribute(), 5.milli)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testAllEventTimeTumblingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Tumble over 2.rows on 'rowtime as 'w)
+      .window(Tumble over 5.milli on 'long as 'w)
       .groupBy('w)
       .select('int.count)
 
@@ -685,27 +571,27 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "int")
+        term("select", "int", "long")
       ),
       term(
         "window",
-        EventTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(), 2.rows)),
+          'long,
+          5.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
     util.verifyTable(windowedTable, expected)
   }
 
-
   @Test
   def testAllProcessingTimeSlidingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Slide over 50.milli every 50.milli as 'w)
+      .window(Slide over 50.milli every 50.milli on 'proctime as 'w)
       .groupBy('w)
       .select('int.count)
 
@@ -714,13 +600,15 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "int")
+        term("select", "int", "proctime")
       ),
       term(
         "window",
-        ProcessingTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          50.milli, 50.milli)),
+          'proctime,
+          50.milli,
+          50.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -730,10 +618,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllProcessingTimeSlidingGroupWindowOverCount(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
 
     val windowedTable = table
-      .window(Slide over 2.rows every 1.rows as 'w)
+      .window(Slide over 2.rows every 1.rows on 'proctime as 'w)
       .groupBy('w)
       .select('int.count)
 
@@ -742,13 +630,15 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "int")
+        term("select", "int", "proctime")
       ),
       term(
         "window",
-        ProcessingTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          2.rows, 1.rows)),
+          'proctime,
+          2.rows,
+          1.rows)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -758,39 +648,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllEventTimeSlidingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Slide over 8.milli every 10.milli on 'rowtime as 'w)
-      .groupBy('w)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term(
-        "window",
-        EventTimeSlidingGroupWindow(
-          WindowReference("w"),
-          RowtimeAttribute(), 8.milli, 10.milli)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testAllEventTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Slide over 2.rows every 1.rows on 'rowtime as 'w)
+      .window(Slide over 8.milli every 10.milli on 'long as 'w)
       .groupBy('w)
       .select('int.count)
 
@@ -799,13 +660,15 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "int")
+        term("select", "int", "long")
       ),
       term(
         "window",
-        EventTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(), 2.rows, 1.rows)),
+          'long,
+          8.milli,
+          10.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -815,10 +678,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllEventTimeSessionGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Session withGap 7.milli on 'rowtime as 'w)
+      .window(Session withGap 7.milli on 'long as 'w)
       .groupBy('w)
       .select('int.count)
 
@@ -827,13 +690,14 @@ class GroupWindowTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
-        term("select", "int")
+        term("select", "int", "long")
       ),
       term(
         "window",
-        EventTimeSessionGroupWindow(
+        SessionGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(), 7.milli)),
+          'long,
+          7.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -843,25 +707,21 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testTumbleWindowStartEnd(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Tumble over 5.milli on 'rowtime as 'w)
+      .window(Tumble over 5.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
     val expected = unaryNode(
       "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
+      streamTableNode(0),
       term("groupBy", "string"),
       term("window",
-        EventTimeTumblingGroupWindow(
+        TumblingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(),
+          'long,
           5.milli)),
       term("select",
         "string",
@@ -876,25 +736,21 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testSlideWindowStartEnd(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
+      .window(Slide over 10.milli every 5.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
     val expected = unaryNode(
       "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
+      streamTableNode(0),
       term("groupBy", "string"),
       term("window",
-        EventTimeSlidingGroupWindow(
+        SlidingGroupWindow(
           WindowReference("w"),
-          RowtimeAttribute(),
+          'long,
           10.milli,
           5.milli)),
       term("select",
@@ -910,10 +766,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testSessionWindowStartWithTwoEnd(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Session withGap 3.milli on 'rowtime as 'w)
+      .window(Session withGap 3.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('w.end as 'we1, 'string, 'int.count as 'cnt, 'w.start as 'ws, 'w.end as 'we2)
 
@@ -921,16 +777,12 @@ class GroupWindowTest extends TableTestBase {
       "DataStreamCalc",
       unaryNode(
         "DataStreamAggregate",
-        unaryNode(
-          "DataStreamCalc",
-          streamTableNode(0),
-          term("select", "string", "int")
-        ),
+        streamTableNode(0),
         term("groupBy", "string"),
         term("window",
-          EventTimeSessionGroupWindow(
+          SessionGroupWindow(
             WindowReference("w"),
-            RowtimeAttribute(),
+            'long,
             3.milli)),
         term("select",
           "string",
@@ -947,10 +799,10 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testTumbleWindowWithDuplicateAggsAndProps(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
-      .window(Tumble over 5.millis on 'rowtime as 'w)
+      .window(Tumble over 5.millis on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.sum + 1 as 's1, 'int.sum + 3 as 's2, 'w.start as 'x, 'w.start as 'x2,
         'w.end as 'x3, 'w.end)
@@ -959,16 +811,12 @@ class GroupWindowTest extends TableTestBase {
       "DataStreamCalc",
       unaryNode(
         "DataStreamAggregate",
-        unaryNode(
-          "DataStreamCalc",
-          streamTableNode(0),
-          term("select", "string", "int")
-        ),
+        streamTableNode(0),
         term("groupBy", "string"),
         term("window",
-          EventTimeTumblingGroupWindow(
+          TumblingGroupWindow(
             WindowReference("w"),
-            RowtimeAttribute(),
+            'long,
             5.millis)),
         term("select",
           "string",

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowITCase.scala
index 4c0fea7..b097767 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowITCase.scala
@@ -58,7 +58,7 @@ class OverWindowITCase extends StreamingWithStateTestBase {
     StreamITCase.testResults = mutable.MutableList()
     StreamITCase.clear
     val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'a, 'b, 'c)
+    val table = stream.toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
     val countFun = new CountAggFunction
     val weightAvgFun = new WeightedAvg
 
@@ -107,7 +107,7 @@ class OverWindowITCase extends StreamingWithStateTestBase {
     )
     val table = env
       .addSource(new RowTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
     val countFun = new CountAggFunction
     val weightAvgFun = new WeightedAvg
 
@@ -173,7 +173,7 @@ class OverWindowITCase extends StreamingWithStateTestBase {
     StreamITCase.testResults = mutable.MutableList()
 
     val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    val table = stream.toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
 
     val windowedTable = table
       .window(Over partitionBy 'a orderBy 'proctime preceding 4.rows following CURRENT_ROW as 'w)
@@ -234,7 +234,8 @@ class OverWindowITCase extends StreamingWithStateTestBase {
     StreamITCase.clear
 
     val table = env.addSource[(Long, Int, String)](
-      new RowTimeSourceFunction[(Long, Int, String)](data)).toTable(tEnv).as('a, 'b, 'c)
+      new RowTimeSourceFunction[(Long, Int, String)](data))
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     val windowedTable = table
       .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows following CURRENT_ROW as 'w)
@@ -295,7 +296,8 @@ class OverWindowITCase extends StreamingWithStateTestBase {
     StreamITCase.clear
 
     val table = env.addSource[(Long, Int, String)](
-      new RowTimeSourceFunction[(Long, Int, String)](data)).toTable(tEnv).as('a, 'b, 'c)
+      new RowTimeSourceFunction[(Long, Int, String)](data))
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     val windowedTable = table
       .window(

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
index 7dea521..ea3ab22 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.table.api.scala.stream.table
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.java.utils.UserDefinedAggFunctions.{WeightedAvgWithRetract}
+import org.apache.flink.table.api.java.utils.UserDefinedAggFunctions.WeightedAvgWithRetract
 import org.apache.flink.table.api.{Table, ValidationException}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.utils.TableTestUtil._
@@ -27,7 +27,8 @@ import org.junit.Test
 
 class OverWindowTest extends TableTestBase {
   private val streamUtil: StreamTableTestUtil = streamTestUtil()
-  val table: Table = streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c)
+  val table: Table = streamUtil.addTable[(Int, String, Long)]("MyTable",
+    'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
 
   @Test(expected = classOf[ValidationException])
   def testInvalidWindowAlias(): Unit = {
@@ -121,12 +122,12 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "b", "c", "PROCTIME() AS $3")
+            term("select", "a", "b", "c", "proctime")
           ),
           term("partitionBy", "b"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "b", "c", "PROCTIME", "WeightedAvgWithRetract(c, a) AS w0$o0")
+          term("select", "a", "b", "c", "proctime", "WeightedAvgWithRetract(c, a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS _c1")
       )
@@ -150,16 +151,16 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
           term("partitionBy", "a"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN 7200000 PRECEDING AND CURRENT ROW"),
           term(
             "select",
             "a",
             "c",
-            "PROCTIME",
+            "proctime",
             "WeightedAvgWithRetract(c, a) AS w0$o0"
           )
         ),
@@ -183,11 +184,11 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN 10000 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(c) AS w0$o0")
+          term("select", "a", "c", "proctime", "COUNT(c) AS w0$o0")
         ),
         term("select", "a", "w0$o0 AS _c1")
       )
@@ -209,11 +210,11 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "proctime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS _c1")
       )
@@ -238,16 +239,16 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
           term(
             "select",
             "a",
             "c",
-            "PROCTIME",
+            "proctime",
             "COUNT(a) AS w0$o0",
             "WeightedAvgWithRetract(c, a) AS w0$o1"
           )
@@ -280,12 +281,12 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME",
+          term("select", "a", "c", "proctime",
                "COUNT(a) AS w0$o0",
                "WeightedAvgWithRetract(c, a) AS w0$o1")
         ),
@@ -310,15 +311,15 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
           term(
             "select",
             "a",
             "c",
-            "PROCTIME",
+            "proctime",
             "COUNT(a) AS w0$o0",
             "SUM(a) AS w0$o1"
           )
@@ -349,11 +350,11 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "proctime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS _c1")
       )
@@ -378,12 +379,12 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "b", "c", "ROWTIME() AS $3")
+            term("select", "a", "b", "c", "rowtime")
           ),
           term("partitionBy", "b"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "b", "c", "ROWTIME",
+          term("select", "a", "b", "c", "rowtime",
                "COUNT(b) AS w0$o0",
                "WeightedAvgWithRetract(c, a) AS w0$o1")
         ),
@@ -410,16 +411,16 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
           term("partitionBy", "a"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN 7200000 PRECEDING AND CURRENT ROW"),
           term(
             "select",
             "a",
             "c",
-            "ROWTIME",
+            "rowtime",
             "AVG(c) AS w0$o0",
             "WeightedAvgWithRetract(c, a) AS w0$o1"
           )
@@ -444,11 +445,11 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN 10000 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(c) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(c) AS w0$o0")
         ),
         term("select", "a", "w0$o0 AS _c1")
       )
@@ -470,11 +471,11 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS _c1")
       )
@@ -499,16 +500,16 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
           term(
             "select",
             "a",
             "c",
-            "ROWTIME",
+            "rowtime",
             "COUNT(a) AS w0$o0",
             "WeightedAvgWithRetract(c, a) AS w0$o1"
           )
@@ -542,12 +543,12 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("rows", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME",
+          term("select", "a", "c", "rowtime",
                "COUNT(a) AS w0$o0",
                "WeightedAvgWithRetract(c, a) AS w0$o1")
         ),
@@ -572,15 +573,15 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
           term(
             "select",
             "a",
             "c",
-            "ROWTIME",
+            "rowtime",
             "COUNT(a) AS w0$o0",
             "SUM(a) AS w0$o1"
           )
@@ -611,11 +612,11 @@ class OverWindowTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("rows", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS _c1")
       )

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/GroupWindowStringExpressionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/GroupWindowStringExpressionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/GroupWindowStringExpressionTest.scala
index d314c9a..d261e36 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/GroupWindowStringExpressionTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/GroupWindowStringExpressionTest.scala
@@ -31,7 +31,7 @@ class GroupWindowStringExpressionTest extends TableTestBase {
   @Test
   def testJavaScalaTableAPIEquality(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Int, Long, String)]('int, 'long, 'string)
+    val t = util.addTable[(Int, Long, String)]('int, 'long, 'string, 'rowtime.rowtime)
 
     val myCountFun = new CountAggFunction
     util.tEnv.registerFunction("myCountFun", myCountFun)
@@ -40,7 +40,7 @@ class GroupWindowStringExpressionTest extends TableTestBase {
 
     // Expression / Scala API
     val resScala = t
-      .window(Slide over 4.rows every 2.rows as 'w)
+      .window(Slide over 4.hours every 2.hours on 'rowtime as 'w)
       .groupBy('w, 'string)
       .select(
         'string,
@@ -51,7 +51,7 @@ class GroupWindowStringExpressionTest extends TableTestBase {
 
     // String / Java API
     val resJava = t
-      .window(JSlide.over("4.rows").every("2.rows").as("w"))
+      .window(JSlide.over("4.hours").every("2.hours").on("rowtime").as("w"))
       .groupBy("w, string")
       .select(
         "string, " +

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/OverWindowStringExpressionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/OverWindowStringExpressionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/OverWindowStringExpressionTest.scala
index 0a5e001..04016f1 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/OverWindowStringExpressionTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/stringexpr/OverWindowStringExpressionTest.scala
@@ -29,7 +29,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testPartitionedUnboundedOverRow(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver partitionBy 'a orderBy 'rowtime preceding UNBOUNDED_ROW as 'w)
@@ -44,7 +44,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testUnboundedOverRow(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver orderBy 'rowtime preceding UNBOUNDED_ROW following CURRENT_ROW as 'w)
@@ -59,7 +59,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testPartitionedBoundedOverRow(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver partitionBy('a, 'd) orderBy 'rowtime preceding 10.rows as 'w)
@@ -74,7 +74,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testBoundedOverRow(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver orderBy 'rowtime preceding 10.rows following CURRENT_ROW as 'w)
@@ -89,7 +89,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testPartitionedUnboundedOverRange(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver partitionBy 'a orderBy 'rowtime preceding UNBOUNDED_RANGE as 'w)
@@ -104,7 +104,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testUnboundedOverRange(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver orderBy 'rowtime preceding UNBOUNDED_RANGE following CURRENT_RANGE as 'w)
@@ -120,7 +120,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testPartitionedBoundedOverRange(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver partitionBy('a, 'c) orderBy 'rowtime preceding 10.minutes as 'w)
@@ -135,7 +135,7 @@ class OverWindowStringExpressionTest extends TableTestBase {
   @Test
   def testBoundedOverRange(): Unit = {
     val util = streamTestUtil()
-    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e)
+    val t = util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rowtime.rowtime)
 
     val resScala = t
       .window(SOver orderBy 'rowtime preceding 4.hours following CURRENT_RANGE as 'w)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamAggregateITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamAggregateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamAggregateITCase.scala
index dcd3c6c..05e1892 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamAggregateITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamAggregateITCase.scala
@@ -62,10 +62,10 @@ class DataStreamAggregateITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
-    val table = stream.toTable(tEnv, 'long, 'int, 'double, 'float, 'bigdec, 'string)
+    val table = stream.toTable(tEnv, 'long.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
 
     val windowedTable = table
-      .window(Slide over 5.milli every 2.milli on 'rowtime as 'w)
+      .window(Slide over 5.milli every 2.milli on 'long as 'w)
       .groupBy('w)
       .select('int.count, 'w.start, 'w.end)
 
@@ -97,10 +97,10 @@ class DataStreamAggregateITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
-    val table = stream.toTable(tEnv, 'long, 'int, 'double, 'float, 'bigdec, 'string)
+    val table = stream.toTable(tEnv, 'long.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
 
     val windowedTable = table
-      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
+      .window(Slide over 10.milli every 5.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
@@ -134,10 +134,10 @@ class DataStreamAggregateITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
-    val table = stream.toTable(tEnv, 'long, 'int, 'double, 'float, 'bigdec, 'string)
+    val table = stream.toTable(tEnv, 'long.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
 
     val windowedTable = table
-      .window(Slide over 5.milli every 4.milli on 'rowtime as 'w)
+      .window(Slide over 5.milli every 4.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
@@ -168,10 +168,10 @@ class DataStreamAggregateITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
-    val table = stream.toTable(tEnv, 'long, 'int, 'double, 'float, 'bigdec, 'string)
+    val table = stream.toTable(tEnv, 'long.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
 
     val windowedTable = table
-      .window(Slide over 5.milli every 10.milli on 'rowtime as 'w)
+      .window(Slide over 5.milli every 10.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
@@ -197,10 +197,10 @@ class DataStreamAggregateITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
-    val table = stream.toTable(tEnv, 'long, 'int, 'double, 'float, 'bigdec, 'string)
+    val table = stream.toTable(tEnv, 'long.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
 
     val windowedTable = table
-      .window(Slide over 3.milli every 10.milli on 'rowtime as 'w)
+      .window(Slide over 3.milli every 10.milli on 'long as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
@@ -225,7 +225,7 @@ class DataStreamAggregateITCase extends StreamingMultipleProgramsTestBase {
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
       .map(t => (t._2, t._6))
-    val table = stream.toTable(tEnv, 'int, 'string)
+    val table = stream.toTable(tEnv, 'int, 'string, 'rowtime.rowtime)
 
     val windowedTable = table
       .window(Slide over 3.milli every 10.milli on 'rowtime as 'w)


[07/15] flink git commit: [FLINK-5884] [table] Integrate time indicators for Table API & SQL. Continued

Posted by fh...@apache.org.
[FLINK-5884] [table] Integrate time indicators for Table API & SQL. Continued


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

Branch: refs/heads/master
Commit: 24bf61ceb332f2db2dc4bab624b73beffae1160a
Parents: 495f104
Author: Fabian Hueske <fh...@apache.org>
Authored: Thu May 4 18:05:27 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:54 2017 +0200

----------------------------------------------------------------------
 .../flink/table/api/BatchTableEnvironment.scala |  25 +--
 .../table/api/StreamTableEnvironment.scala      |  86 ++++++++--
 .../flink/table/api/TableEnvironment.scala      |  64 +-------
 .../flink/table/calcite/FlinkTypeFactory.scala  |  17 +-
 .../table/expressions/ExpressionParser.scala    |  18 +-
 .../table/plan/logical/LogicalWindow.scala      |   2 +-
 .../flink/table/plan/logical/groupWindows.scala |  11 +-
 .../flink/table/plan/nodes/CommonCalc.scala     |  10 +-
 .../plan/nodes/PhysicalTableSourceScan.scala    |   2 +-
 .../plan/nodes/dataset/DataSetAggregate.scala   |   2 +-
 .../table/plan/nodes/dataset/DataSetCalc.scala  |   5 +-
 .../datastream/DataStreamOverAggregate.scala    |   5 +-
 .../datastream/StreamTableSourceScan.scala      |  28 +---
 .../logical/FlinkLogicalTableSourceScan.scala   |  32 +++-
 .../plan/schema/StreamTableSourceTable.scala    |  65 ++++++++
 .../table/runtime/aggregate/AggregateUtil.scala |   3 +-
 .../table/sources/DefinedTimeAttributes.scala   |  47 ++++--
 .../flink/table/TableEnvironmentTest.scala      |  52 ++----
 .../api/scala/batch/table/GroupWindowTest.scala |   4 +-
 .../stream/StreamTableEnvironmentTest.scala     | 164 +++++++++++++++++++
 .../api/scala/stream/TableSourceTest.scala      | 154 +++++++++++++++++
 .../scala/stream/sql/WindowAggregateTest.scala  |  59 ++++---
 22 files changed, 628 insertions(+), 227 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index 3eb2ffc..02c6063 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -31,7 +31,7 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
 import org.apache.flink.table.explain.PlanJsonParser
-import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.expressions.{Expression, RowtimeAttribute, TimeAttribute}
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.dataset.DataSetRel
 import org.apache.flink.table.plan.rules.FlinkRuleSets
@@ -196,26 +196,11 @@ abstract class BatchTableEnvironment(
 
     val (fieldNames, fieldIndexes) = getFieldInfo[T](
       dataSet.getType,
-      fields,
-      ignoreTimeAttributes = true)
+      fields)
 
-    // validate and extract time attributes
-    val (rowtime, proctime) = validateAndExtractTimeAttributes(fieldNames, fieldIndexes, fields)
-
-    // don't allow proctime on batch
-    proctime match {
-      case Some(_) =>
-        throw new ValidationException(
-          "A proctime attribute is not allowed in a batch environment. " +
-            "Working with processing-time on batch would lead to non-deterministic results.")
-      case _ => // ok
-    }
-    // rowtime must not extend the schema of a batch table
-    rowtime match {
-      case Some((idx, _)) if idx >= dataSet.getType.getArity =>
-        throw new ValidationException(
-          "A rowtime attribute must be defined on an existing field in a batch environment.")
-      case _ => // ok
+    if (fields.exists(_.isInstanceOf[TimeAttribute])) {
+      throw new ValidationException(
+        ".rowtime and .proctime time indicators are not allowed in a batch environment.")
     }
 
     val dataSetTable = new DataSetTable[T](dataSet, fieldIndexes, fieldNames)

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index d1f2fb5..dd2c09d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -26,19 +26,21 @@ import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql2rel.RelDecorrelator
 import org.apache.calcite.tools.{RuleSet, RuleSets}
-import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.table.calcite.RelTimeIndicatorConverter
 import org.apache.flink.table.explain.PlanJsonParser
-import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.expressions.{Expression, ProctimeAttribute, RowtimeAttribute, UnresolvedFieldReference}
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
 import org.apache.flink.table.plan.rules.FlinkRuleSets
-import org.apache.flink.table.plan.schema.{DataStreamTable, TableSourceTable}
+import org.apache.flink.table.plan.schema.{DataStreamTable, StreamTableSourceTable, TableSourceTable}
 import org.apache.flink.table.sinks.{StreamTableSink, TableSink}
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
+import org.apache.flink.table.typeutils.TypeCheckUtils
 import org.apache.flink.types.Row
 
 import _root_.scala.collection.JavaConverters._
@@ -99,7 +101,7 @@ abstract class StreamTableEnvironment(
 
     tableSource match {
       case streamTableSource: StreamTableSource[_] =>
-        registerTableInternal(name, new TableSourceTable(streamTableSource))
+        registerTableInternal(name, new StreamTableSourceTable(streamTableSource))
       case _ =>
         throw new TableException("Only StreamTableSource can be registered in " +
             "StreamTableEnvironment")
@@ -168,14 +170,13 @@ abstract class StreamTableEnvironment(
       fields: Array[Expression])
     : Unit = {
 
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](
-      dataStream.getType,
-      fields,
-      ignoreTimeAttributes = false)
+    val streamType = dataStream.getType
 
-    // validate and extract time attributes
-    val (rowtime, proctime) = validateAndExtractTimeAttributes(fieldNames, fieldIndexes, fields)
+    // get field names and types for all non-replaced fields
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](streamType, fields)
 
+    // validate and extract time attributes
+    val (rowtime, proctime) = validateAndExtractTimeAttributes(streamType, fields)
 
     val dataStreamTable = new DataStreamTable[T](
       dataStream,
@@ -188,6 +189,71 @@ abstract class StreamTableEnvironment(
   }
 
   /**
+    * Checks for at most one rowtime and proctime attribute.
+    * Returns the time attributes.
+    *
+    * @return rowtime attribute and proctime attribute
+    */
+  private def validateAndExtractTimeAttributes(
+    streamType: TypeInformation[_],
+    exprs: Array[Expression])
+  : (Option[(Int, String)], Option[(Int, String)]) = {
+
+    val fieldTypes: Array[TypeInformation[_]] = streamType match {
+      case c: CompositeType[_] => (0 until c.getArity).map(i => c.getTypeAt(i)).toArray
+      case a: AtomicType[_] => Array(a)
+    }
+
+    var fieldNames: List[String] = Nil
+    var rowtime: Option[(Int, String)] = None
+    var proctime: Option[(Int, String)] = None
+
+    exprs.zipWithIndex.foreach {
+      case (RowtimeAttribute(reference@UnresolvedFieldReference(name)), idx) =>
+        if (rowtime.isDefined) {
+          throw new TableException(
+            "The rowtime attribute can only be defined once in a table schema.")
+        } else {
+          // check type of field that is replaced
+          if (idx < fieldTypes.length &&
+            !(TypeCheckUtils.isLong(fieldTypes(idx)) ||
+              TypeCheckUtils.isTimePoint(fieldTypes(idx)))) {
+            throw new TableException(
+              "The rowtime attribute can only be replace a field with a valid time type, such as " +
+                "Timestamp or Long.")
+          }
+          rowtime = Some(idx, name)
+        }
+      case (ProctimeAttribute(reference@UnresolvedFieldReference(name)), idx) =>
+        if (proctime.isDefined) {
+          throw new TableException(
+            "The proctime attribute can only be defined once in a table schema.")
+        } else {
+          // check that proctime is only appended
+          if (idx < fieldTypes.length) {
+            throw new TableException(
+              "The proctime attribute can only be appended to the table schema and not replace " +
+                "an existing field. Please move it to the end of the schema.")
+          }
+          proctime = Some(idx, name)
+        }
+      case (u: UnresolvedFieldReference, _) => fieldNames = u.name :: fieldNames
+    }
+
+    if (rowtime.isDefined && fieldNames.contains(rowtime.get._2)) {
+      throw new TableException(
+        "The rowtime attribute may not have the same name as an another field.")
+    }
+
+    if (proctime.isDefined && fieldNames.contains(proctime.get._2)) {
+      throw new TableException(
+        "The proctime attribute may not have the same name as an another field.")
+    }
+
+    (rowtime, proctime)
+  }
+
+  /**
     * Returns the decoration rule set for this environment
     * including a custom RuleSet configuration.
     */

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 4c72e8f..9ed5000 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -601,50 +601,36 @@ abstract class TableEnvironment(val config: TableConfig) {
 
   /**
     * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
-    * [[Expression]]. It does not handle time attributes but considers them in indices, if
-    * ignore flag is not false.
+    * [[Expression]]. It does not handle time attributes but considers them in indices.
     *
     * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
-    * @param exprs The expressions that define the field names.
-    * @param ignoreTimeAttributes ignore time attributes and handle them as regular expressions.
+    * @param exprs     The expressions that define the field names.
     * @tparam A The type of the TypeInformation.
     * @return A tuple of two arrays holding the field names and corresponding field positions.
     */
   protected[flink] def getFieldInfo[A](
       inputType: TypeInformation[A],
-      exprs: Array[Expression],
-      ignoreTimeAttributes: Boolean)
+      exprs: Array[Expression])
     : (Array[String], Array[Int]) = {
 
     TableEnvironment.validateType(inputType)
 
-    val filteredExprs = if (ignoreTimeAttributes) {
-        exprs.map {
-          case ta: TimeAttribute => ta.expression
-          case e@_ => e
-        }
-    } else {
-      exprs
-    }
-
     val indexedNames: Array[(Int, String)] = inputType match {
       case g: GenericTypeInfo[A] if g.getTypeClass == classOf[Row] =>
         throw new TableException(
           "An input of GenericTypeInfo<Row> cannot be converted to Table. " +
             "Please specify the type of the input with a RowTypeInfo.")
       case a: AtomicType[A] =>
-        filteredExprs.zipWithIndex flatMap {
+        exprs.zipWithIndex flatMap {
           case (UnresolvedFieldReference(name), idx) =>
             if (idx > 0) {
               throw new TableException("Table of atomic type can only have a single field.")
             }
             Some((0, name))
-          case (_: TimeAttribute, _) if ignoreTimeAttributes =>
-            None
           case _ => throw new TableException("Field reference expression requested.")
         }
       case t: TupleTypeInfo[A] =>
-        filteredExprs.zipWithIndex flatMap {
+        exprs.zipWithIndex flatMap {
           case (UnresolvedFieldReference(name), idx) =>
             Some((idx, name))
           case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
@@ -659,7 +645,7 @@ abstract class TableEnvironment(val config: TableConfig) {
             "Field reference expression or alias on field expression expected.")
         }
       case c: CaseClassTypeInfo[A] =>
-        filteredExprs.zipWithIndex flatMap {
+        exprs.zipWithIndex flatMap {
           case (UnresolvedFieldReference(name), idx) =>
             Some((idx, name))
           case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
@@ -674,7 +660,7 @@ abstract class TableEnvironment(val config: TableConfig) {
             "Field reference expression or alias on field expression expected.")
         }
       case p: PojoTypeInfo[A] =>
-        filteredExprs flatMap {
+        exprs flatMap {
           case (UnresolvedFieldReference(name)) =>
             val idx = p.getFieldIndex(name)
             if (idx < 0) {
@@ -822,42 +808,6 @@ abstract class TableEnvironment(val config: TableConfig) {
     Some(mapFunction)
   }
 
-  /**
-    * Checks for at most one rowtime and proctime attribute.
-    * Returns the time attributes.
-    *
-    * @return rowtime attribute and proctime attribute
-    */
-  protected def validateAndExtractTimeAttributes(
-      fieldNames: Seq[String],
-      fieldIndices: Seq[Int],
-      exprs: Array[Expression])
-    : (Option[(Int, String)], Option[(Int, String)]) = {
-
-    var rowtime: Option[(Int, String)] = None
-    var proctime: Option[(Int, String)] = None
-
-    exprs.zipWithIndex.foreach {
-      case (RowtimeAttribute(reference@UnresolvedFieldReference(name)), idx) =>
-        if (rowtime.isDefined) {
-          throw new TableException(
-            "The rowtime attribute can only be defined once in a table schema.")
-        } else {
-          rowtime = Some(idx, name)
-        }
-      case (ProctimeAttribute(reference@UnresolvedFieldReference(name)), idx) =>
-        if (proctime.isDefined) {
-          throw new TableException(
-            "The proctime attribute can only be defined once in a table schema.")
-        } else {
-          proctime = Some(idx, name)
-        }
-      case _ =>
-        // do nothing
-    }
-
-    (rowtime, proctime)
-  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
index 001011b..9281ad8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
@@ -65,11 +65,12 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp
           createSqlIntervalType(
             new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO))
 
-        case TimeIndicatorTypeInfo.ROWTIME_INDICATOR =>
-          createRowtimeIndicatorType()
-
-        case TimeIndicatorTypeInfo.PROCTIME_INDICATOR =>
-          createProctimeIndicatorType()
+        case TIMESTAMP if typeInfo.isInstanceOf[TimeIndicatorTypeInfo] =>
+          if (typeInfo.asInstanceOf[TimeIndicatorTypeInfo].isEventTime) {
+            createRowtimeIndicatorType()
+          } else {
+            createProctimeIndicatorType()
+          }
 
         case _ =>
           createSqlType(sqlType)
@@ -114,9 +115,11 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp
     * @param fieldNames field names
     * @param fieldTypes field types, every element is Flink's [[TypeInformation]]
     * @param rowtime optional system field to indicate event-time; the index determines the index
-    *                in the final record and might replace an existing field
+    *                in the final record. If the index is smaller than the number of specified
+    *                fields, it shifts all following fields.
     * @param proctime optional system field to indicate processing-time; the index determines the
-    *                 index in the final record and might replace an existing field
+    *                 index in the final record. If the index is smaller than the number of
+    *                 specified fields, it shifts all following fields.
     * @return a struct type with the input fieldNames, input fieldTypes, and system fields
     */
   def buildLogicalRowType(

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
index c33f8fc..98580ba 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
@@ -98,11 +98,14 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {
   lazy val UNBOUNDED_ROW: Keyword = Keyword("unbounded_row")
   lazy val UNBOUNDED_RANGE: Keyword = Keyword("unbounded_range")
   lazy val ASIN: Keyword = Keyword("asin")
+  lazy val ROWTIME: Keyword = Keyword("rowtime")
+  lazy val PROCTIME: Keyword = Keyword("proctime")
 
   def functionIdent: ExpressionParser.Parser[String] =
     not(ARRAY) ~ not(AS) ~ not(COUNT) ~ not(AVG) ~ not(MIN) ~ not(MAX) ~
       not(SUM0) ~ not(STDDEV_POP) ~ not(STDDEV_SAMP) ~ not(VAR_POP) ~ not(VAR_SAMP) ~
       not(SUM) ~ not(START) ~ not(END)~ not(CAST) ~ not(NULL) ~ not(IF) ~
+      not(ROWTIME) ~ not(PROCTIME) ~
       not(CURRENT_ROW) ~ not(UNBOUNDED_ROW) ~ not(CURRENT_RANGE) ~ not(UNBOUNDED_RANGE) ~>
       super.ident
 
@@ -532,12 +535,25 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {
 
   // alias
 
-  lazy val alias: PackratParser[Expression] = logic ~ AS ~ fieldReference ^^ {
+  lazy val alias: PackratParser[Expression] = timeIndicator |
+    logic ~ AS ~ fieldReference ^^ {
       case e ~ _ ~ name => Alias(e, name.name)
   } | logic ~ AS ~ "(" ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
     case e ~ _ ~ _ ~ names ~ _ => Alias(e, names.head.name, names.tail.map(_.name))
   } | logic
 
+  // time indicators
+
+  lazy val timeIndicator: PackratParser[Expression] = procTime | rowTime
+
+  lazy val procTime: PackratParser[Expression] = fieldReference ~ "." ~ PROCTIME ^^ {
+    case f ~ _ ~ _ => ProctimeAttribute(f)
+  }
+
+  lazy val rowTime: PackratParser[Expression] = fieldReference ~ "." ~ ROWTIME ^^ {
+    case f ~ _ ~ _ => RowtimeAttribute(f)
+  }
+
   lazy val expression: PackratParser[Expression] = alias |
     failure("Invalid expression.")
 

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
index 92dc501..6161ef0 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.expressions.{Expression, WindowReference}
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
 
 /**
-  * Logical super class for all types of windows (group-windows and row-windows).
+  * Logical super class for group windows.
   *
   * @param aliasAttribute window alias
   * @param timeAttribute time field indicating event-time or processing-time

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
index 3e5de28..4a8fb52 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.logical
 import org.apache.flink.table.api.{BatchTableEnvironment, StreamTableEnvironment, TableEnvironment}
 import org.apache.flink.table.expressions.ExpressionUtils.{isRowCountLiteral, isRowtimeAttribute, isTimeAttribute, isTimeIntervalLiteral}
 import org.apache.flink.table.expressions._
-import org.apache.flink.table.typeutils.TypeCheckUtils.isTimePoint
+import org.apache.flink.table.typeutils.TypeCheckUtils.{isTimePoint, isLong}
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
 
 // ------------------------------------------------------------------------------------------------
@@ -56,7 +56,8 @@ case class TumblingGroupWindow(
         case _: StreamTableEnvironment if !isTimeAttribute(timeField) =>
           ValidationFailure(
             "Tumbling window expects a time attribute for grouping in a stream environment.")
-        case _: BatchTableEnvironment if isTimePoint(size.resultType) =>
+        case _: BatchTableEnvironment
+          if !(isTimePoint(timeField.resultType) || isLong(timeField.resultType)) =>
           ValidationFailure(
             "Tumbling window expects a time attribute for grouping in a stream environment.")
 
@@ -119,7 +120,8 @@ case class SlidingGroupWindow(
         case _: StreamTableEnvironment if !isTimeAttribute(timeField) =>
           ValidationFailure(
             "Sliding window expects a time attribute for grouping in a stream environment.")
-        case _: BatchTableEnvironment if isTimePoint(size.resultType) =>
+        case _: BatchTableEnvironment
+          if !(isTimePoint(timeField.resultType) || isLong(timeField.resultType)) =>
           ValidationFailure(
             "Sliding window expects a time attribute for grouping in a stream environment.")
 
@@ -169,7 +171,8 @@ case class SessionGroupWindow(
         case _: StreamTableEnvironment if !isTimeAttribute(timeField) =>
           ValidationFailure(
             "Session window expects a time attribute for grouping in a stream environment.")
-        case _: BatchTableEnvironment if isTimePoint(gap.resultType) =>
+        case _: BatchTableEnvironment
+          if !(isTimePoint(timeField.resultType) || isLong(timeField.resultType)) =>
           ValidationFailure(
             "Session window expects a time attribute for grouping in a stream environment.")
 

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
index 5c35129..ff5ffb2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
@@ -34,11 +34,11 @@ import scala.collection.JavaConverters._
 trait CommonCalc {
 
   private[flink] def functionBody(
-     generator: CodeGenerator,
-     inputSchema: RowSchema,
-     returnSchema: RowSchema,
-     calcProgram: RexProgram,
-     config: TableConfig)
+      generator: CodeGenerator,
+      inputSchema: RowSchema,
+      returnSchema: RowSchema,
+      calcProgram: RexProgram,
+      config: TableConfig)
     : String = {
 
     val expandedExpressions = calcProgram

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
index c18c3d1..dc7a0d6 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
@@ -46,7 +46,7 @@ abstract class PhysicalTableSourceScan(
 
   override def explainTerms(pw: RelWriter): RelWriter = {
     val terms = super.explainTerms(pw)
-        .item("fields", TableEnvironment.getFieldNames(tableSource).mkString(", "))
+        .item("fields", deriveRowType().getFieldNames.asScala.mkString(", "))
 
     val sourceDesc = tableSource.explainSource()
     if (sourceDesc.nonEmpty) {

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
index c22dc54..b53081c 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
@@ -90,8 +90,8 @@ class DataSetAggregate(
 
   override def translateToPlan(tableEnv: BatchTableEnvironment): DataSet[Row] = {
 
-    val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
     val input = inputNode.asInstanceOf[DataSetRel]
+    val inputDS = input.translateToPlan(tableEnv)
 
     val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
 

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
index 9e18082..5274fa1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
@@ -103,10 +103,7 @@ class DataSetCalc(
       body,
       rowTypeInfo)
 
-    val runner = new FlatMapRunner[Row, Row](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
+    val runner = calcMapFunction(genFunction)
 
     inputDS.flatMap(runner).name(calcOpName(calcProgram, getExpressionString))
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 8eb9d40..db31f32 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -241,10 +241,7 @@ class DataStreamOverAggregate(
     }
 
     val precedingOffset =
-      getLowerBoundary(
-        logicWindow,
-        overWindow,
-        input) + (if (isRowsClause) 1 else 0)
+      getLowerBoundary(logicWindow, overWindow, input) + (if (isRowsClause) 1 else 0)
 
     val processFunction = AggregateUtil.createBoundedOverProcessFunction(
       generator,

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
index 5dc3da8..e34e416 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
@@ -26,7 +26,7 @@ import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.PhysicalTableSourceScan
 import org.apache.flink.table.plan.schema.{RowSchema, TableSourceTable}
-import org.apache.flink.table.sources.{DefinedTimeAttributes, StreamTableSource, TableSource}
+import org.apache.flink.table.sources._
 import org.apache.flink.types.Row
 
 /** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */
@@ -41,35 +41,23 @@ class StreamTableSourceScan(
   override def deriveRowType() = {
     val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
 
-    def removeIndex[T](idx: Int, l: List[T]): List[T] = {
-      if (l.size < idx) {
-        l
-      } else {
-        l.take(idx) ++ l.drop(idx + 1)
-      }
-    }
+    val fieldNames = TableEnvironment.getFieldNames(tableSource).toList
+    val fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList
 
-    var fieldNames = TableEnvironment.getFieldNames(tableSource).toList
-    var fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList
+    val fieldCnt = fieldNames.length
 
     val rowtime = tableSource match {
-      case timeSource: DefinedTimeAttributes if timeSource.getRowtimeAttribute != null =>
+      case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
         val rowtimeAttribute = timeSource.getRowtimeAttribute
-        // remove physical field if it is overwritten by time attribute
-        fieldNames = removeIndex(rowtimeAttribute.f0, fieldNames)
-        fieldTypes = removeIndex(rowtimeAttribute.f0, fieldTypes)
-        Some((rowtimeAttribute.f0, rowtimeAttribute.f1))
+        Some((fieldCnt, rowtimeAttribute))
       case _ =>
         None
     }
 
     val proctime = tableSource match {
-      case timeSource: DefinedTimeAttributes if timeSource.getProctimeAttribute != null =>
+      case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
         val proctimeAttribute = timeSource.getProctimeAttribute
-        // remove physical field if it is overwritten by time attribute
-        fieldNames = removeIndex(proctimeAttribute.f0, fieldNames)
-        fieldTypes = removeIndex(proctimeAttribute.f0, fieldTypes)
-        Some((proctimeAttribute.f0, proctimeAttribute.f1))
+        Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
       case _ =>
         None
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
index 53e7b31..a2777ec 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
@@ -29,7 +29,7 @@ import org.apache.flink.table.api.TableEnvironment
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.schema.TableSourceTable
-import org.apache.flink.table.sources.TableSource
+import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, TableSource}
 
 import scala.collection.JavaConverters._
 
@@ -47,11 +47,33 @@ class FlinkLogicalTableSourceScan(
 
   override def deriveRowType(): RelDataType = {
     val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+
+    val fieldNames = TableEnvironment.getFieldNames(tableSource).toList
+    val fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList
+
+    val fieldCnt = fieldNames.length
+
+    val rowtime = tableSource match {
+      case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+        val rowtimeAttribute = timeSource.getRowtimeAttribute
+        Some((fieldCnt, rowtimeAttribute))
+      case _ =>
+        None
+    }
+
+    val proctime = tableSource match {
+      case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+        val proctimeAttribute = timeSource.getProctimeAttribute
+        Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
+      case _ =>
+        None
+    }
+
     flinkTypeFactory.buildLogicalRowType(
-      TableEnvironment.getFieldNames(tableSource),
-      TableEnvironment.getFieldTypes(tableSource.getReturnType),
-      None,
-      None)
+      fieldNames,
+      fieldTypes,
+      rowtime,
+      proctime)
   }
 
   override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
new file mode 100644
index 0000000..75deca5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
@@ -0,0 +1,65 @@
+/*
+ * 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.table.plan.schema
+
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, TableSource}
+
+class StreamTableSourceTable[T](
+    override val tableSource: TableSource[T],
+    override val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN)
+  extends TableSourceTable[T](tableSource, statistic) {
+
+
+  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = {
+    val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
+
+    val fieldNames = TableEnvironment.getFieldNames(tableSource).toList
+    val fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList
+
+    val fieldCnt = fieldNames.length
+
+    val rowtime = tableSource match {
+      case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+        val rowtimeAttribute = timeSource.getRowtimeAttribute
+        Some((fieldCnt, rowtimeAttribute))
+      case _ =>
+        None
+    }
+
+    val proctime = tableSource match {
+      case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+        val proctimeAttribute = timeSource.getProctimeAttribute
+        Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
+      case _ =>
+        None
+    }
+
+    flinkTypeFactory.buildLogicalRowType(
+      fieldNames,
+      fieldTypes,
+      rowtime,
+      proctime)
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index 07992cd..dfed34a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -81,7 +81,6 @@ object AggregateUtil {
       isRowsClause: Boolean)
     : ProcessFunction[Row, Row] = {
 
-    val needRetract = false
     val (aggFields, aggregates) =
       transformToAggregateFunctions(
         namedAggregates.map(_.getKey),
@@ -107,7 +106,7 @@ object AggregateUtil {
       None,
       None,
       outputArity,
-      needRetract,
+      needRetract = false,
       needMerge = false,
       needReset = false
     )

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
index 8466cdf..6d87663 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
@@ -18,30 +18,43 @@
 
 package org.apache.flink.table.sources
 
-import org.apache.flink.api.java.tuple.Tuple2
-
 /**
-  * Defines logical time attributes for a [[TableSource]]. Time attributes can be used for
-  * indicating, accessing, and working with Flink's event-time or processing-time. A
-  * [[TableSource]] that implements this interface can define names and positions of rowtime
-  * and proctime attributes in the rows it produces.
+  * Defines a logical event-time attribute for a [[TableSource]].
+  * The event-time attribute can be used for indicating, accessing, and working with Flink's
+  * event-time.
+  *
+  * A [[TableSource]] that implements this interface defines the name of
+  * the event-time attribute. The attribute will be added to the schema of the
+  * [[org.apache.flink.table.api.Table]] produced by the [[TableSource]].
   */
-trait DefinedTimeAttributes {
+trait DefinedRowTimeAttribute {
 
   /**
-    * Defines a name and position (starting at 0) of rowtime attribute that represents Flink's
-    * event-time. Null if no rowtime should be available. If the position is within the arity of
-    * the result row, the logical attribute will overwrite the physical attribute. If the position
-    * is higher than the result row, the time attribute will be appended logically.
+    * Defines a name of the event-time attribute that represents Flink's
+    * event-time. Null if no rowtime should be available.
+    *
+    * The field will be appended to the schema provided by the [[TableSource]].
     */
-  def getRowtimeAttribute: Tuple2[Int, String]
+  def getRowtimeAttribute: String
+}
+
+/**
+  * Defines a logical processing-time attribute for a [[TableSource]].
+  * The processing-time attribute can be used for indicating, accessing, and working with Flink's
+  * processing-time.
+  *
+  * A [[TableSource]] that implements this interface defines the name of
+  * the processing-time attribute. The attribute will be added to the schema of the
+  * [[org.apache.flink.table.api.Table]] produced by the [[TableSource]].
+  */
+trait DefinedProcTimeAttribute {
 
   /**
-    * Defines a name and position (starting at 0) of proctime attribute that represents Flink's
-    * processing-time. Null if no proctime should be available. If the position is within the arity
-    * of the result row, the logical attribute will overwrite the physical attribute. If the
-    * position is higher than the result row, the time attribute will be appended logically.
+    * Defines a name of the processing-time attribute that represents Flink's
+    * processing-time. Null if no rowtime should be available.
+    *
+    * The field will be appended to the schema provided by the [[TableSource]].
     */
-  def getProctimeAttribute: Tuple2[Int, String]
+  def getProctimeAttribute: String
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
index faacc54..5247685 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
@@ -93,8 +93,7 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2"),
         UnresolvedFieldReference("name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -108,8 +107,7 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2"),
         UnresolvedFieldReference("name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -123,8 +121,7 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2"),
         UnresolvedFieldReference("name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
   }
 
   @Test
@@ -135,8 +132,7 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("pf3"),
         UnresolvedFieldReference("pf1"),
         UnresolvedFieldReference("pf2")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("pf3", "pf1", "pf2")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -146,9 +142,7 @@ class TableEnvironmentTest extends TableTestBase {
   def testGetFieldInfoAtomicName1(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(
       atomicType,
-      Array(UnresolvedFieldReference("name")),
-      ignoreTimeAttributes = true
-    )
+      Array(UnresolvedFieldReference("name")))
 
     fieldInfo._1.zip(Array("name")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0)).foreach(x => assertEquals(x._2, x._1))
@@ -161,8 +155,7 @@ class TableEnvironmentTest extends TableTestBase {
       Array(
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2")
-      ),
-      ignoreTimeAttributes = true)
+      ))
   }
 
   @Test
@@ -173,8 +166,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("f0"), "name1"),
         Alias(UnresolvedFieldReference("f1"), "name2"),
         Alias(UnresolvedFieldReference("f2"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -188,8 +180,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("f2"), "name1"),
         Alias(UnresolvedFieldReference("f0"), "name2"),
         Alias(UnresolvedFieldReference("f1"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -203,8 +194,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("xxx"), "name1"),
         Alias(UnresolvedFieldReference("yyy"), "name2"),
         Alias(UnresolvedFieldReference("zzz"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
   }
 
   @Test
@@ -215,8 +205,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("cf1"), "name1"),
         Alias(UnresolvedFieldReference("cf2"), "name2"),
         Alias(UnresolvedFieldReference("cf3"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -230,8 +219,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("cf3"), "name1"),
         Alias(UnresolvedFieldReference("cf1"), "name2"),
         Alias(UnresolvedFieldReference("cf2"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -245,8 +233,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("xxx"), "name1"),
         Alias(UnresolvedFieldReference("yyy"), "name2"),
         Alias(UnresolvedFieldReference("zzz"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
   }
 
   @Test
@@ -257,8 +244,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("pf1"), "name1"),
         Alias(UnresolvedFieldReference("pf2"), "name2"),
         Alias(UnresolvedFieldReference("pf3"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -272,8 +258,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("pf3"), "name1"),
         Alias(UnresolvedFieldReference("pf1"), "name2"),
         Alias(UnresolvedFieldReference("pf2"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -287,8 +272,7 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("xxx"), "name1"),
         Alias(UnresolvedFieldReference("yyy"), "name2"),
         Alias(UnresolvedFieldReference("zzz"), "name3")
-      ),
-      ignoreTimeAttributes = true)
+      ))
   }
 
   @Test(expected = classOf[TableException])
@@ -297,16 +281,14 @@ class TableEnvironmentTest extends TableTestBase {
       atomicType,
       Array(
         Alias(UnresolvedFieldReference("name1"), "name2")
-      ),
-      ignoreTimeAttributes = true)
+      ))
   }
 
   @Test(expected = classOf[TableException])
   def testGetFieldInfoGenericRowAlias(): Unit = {
     tEnv.getFieldInfo(
       genericRowType,
-      Array(UnresolvedFieldReference("first")),
-      ignoreTimeAttributes = true)
+      Array(UnresolvedFieldReference("first")))
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
index c481105..aa6edd3 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
@@ -100,7 +100,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeTumblingGroupWindowOverCount(): Unit = {
     val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
 
     val windowedTable = table
       .window(Tumble over 2.rows on 'long as 'w)
@@ -144,7 +144,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeTumblingGroupWindowOverTime(): Unit = {
     val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
 
     val windowedTable = table
       .window(Tumble over 5.milli on 'long as 'w)

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
new file mode 100644
index 0000000..e9384c7
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
@@ -0,0 +1,164 @@
+/*
+ * 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.table.api.scala.stream
+
+import java.lang.{Integer => JInt, Long => JLong}
+import java.util.Collections
+import java.util.{List => JList}
+
+import org.apache.flink.api.java.tuple.{Tuple5 => JTuple5}
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JStreamExecEnv}
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.{TableEnvironment, TableException, Types}
+import org.apache.flink.table.api.java.{StreamTableEnvironment => JStreamTableEnv}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestBase
+import org.junit.Test
+import org.mockito.Mockito.{mock, when}
+
+class StreamTableEnvironmentTest extends TableTestBase {
+
+  @Test(expected = classOf[TableException])
+  def testInvalidProctimeAttribute(): Unit = {
+    val util = streamTestUtil()
+    // cannot replace an attribute with proctime
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b.proctime, 'c, 'd, 'e)
+  }
+
+  @Test
+  def testProctimeAttribute(): Unit = {
+    val util = streamTestUtil()
+    // cannot replace an attribute with proctime
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'pt.proctime)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testRowtimeAttributeReplaceFieldOfInvalidType(): Unit = {
+    val util = streamTestUtil()
+    // cannot replace a non-time attribute with rowtime
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c.rowtime, 'd, 'e)
+  }
+
+  @Test
+  def testReplacedRowtimeAttribute(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('rt.rowtime, 'b, 'c, 'd, 'e)
+  }
+
+  @Test
+  def testAppendedRowtimeAttribute(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rt.rowtime)
+  }
+
+  @Test
+  def testRowtimeAndProctimeAttribute1(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'rt.rowtime, 'pt.proctime)
+  }
+
+  @Test
+  def testRowtimeAndProctimeAttribute2(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'pt.proctime, 'rt.rowtime)
+  }
+
+  @Test
+  def testRowtimeAndProctimeAttribute3(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('rt.rowtime, 'b, 'c, 'd, 'e, 'pt.proctime)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testRowtimeAndInvalidProctimeAttribute(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('rt.rowtime, 'b, 'c, 'd, 'pt.proctime)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testOnlyOneRowtimeAttribute1(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a.rowtime, 'b, 'c, 'd, 'e, 'rt.rowtime)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testOnlyOneProctimeAttribute1(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'pt1.proctime, 'pt2.proctime)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testRowtimeAttributeUsedName(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'a.rowtime)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testProctimeAttributeUsedName(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int, String, Int, Long)]('a, 'b, 'c, 'd, 'e, 'b.proctime)
+  }
+
+  @Test
+  def testProctimeAttributeParsed(): Unit = {
+    val (jTEnv, ds) = prepareSchemaExpressionParser
+    jTEnv.fromDataStream(ds, "a, b, c, d, e, pt.proctime")
+  }
+
+  @Test
+  def testReplacingRowtimeAttributeParsed(): Unit = {
+    val (jTEnv, ds) = prepareSchemaExpressionParser
+    jTEnv.fromDataStream(ds, "a.rowtime, b, c, d, e")
+  }
+
+  @Test
+  def testAppedingRowtimeAttributeParsed(): Unit = {
+    val (jTEnv, ds) = prepareSchemaExpressionParser
+    jTEnv.fromDataStream(ds, "a, b, c, d, e, rt.rowtime")
+  }
+
+  @Test
+  def testRowtimeAndProctimeAttributeParsed1(): Unit = {
+    val (jTEnv, ds) = prepareSchemaExpressionParser
+    jTEnv.fromDataStream(ds, "a, b, c, d, e, pt.proctime, rt.rowtime")
+  }
+
+  @Test
+  def testRowtimeAndProctimeAttributeParsed2(): Unit = {
+    val (jTEnv, ds) = prepareSchemaExpressionParser
+    jTEnv.fromDataStream(ds, "rt.rowtime, b, c, d, e, pt.proctime")
+  }
+
+  private def prepareSchemaExpressionParser:
+    (JStreamTableEnv, DataStream[JTuple5[JLong, JInt, String, JInt, JLong]]) = {
+
+    val jTEnv = TableEnvironment.getTableEnvironment(mock(classOf[JStreamExecEnv]))
+
+    val sType = new TupleTypeInfo(Types.LONG, Types.INT, Types.STRING, Types.INT, Types.LONG)
+      .asInstanceOf[TupleTypeInfo[JTuple5[JLong, JInt, String, JInt, JLong]]]
+    val ds = mock(classOf[DataStream[JTuple5[JLong, JInt, String, JInt, JLong]]])
+    when(ds.getType).thenReturn(sType)
+
+    (jTEnv, ds)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
new file mode 100644
index 0000000..7673266
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
@@ -0,0 +1,154 @@
+/*
+ * 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.table.api.scala.stream
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, StreamTableSource}
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil.{term, unaryNode}
+import org.apache.flink.types.Row
+import org.junit.Test
+
+class TableSourceTest extends TableTestBase {
+
+  @Test
+  def testRowTimeTableSourceSimple(): Unit = {
+    val util = streamTestUtil()
+    util.tEnv.registerTableSource("rowTimeT", new TestRowTimeSource("addTime"))
+
+    val t = util.tEnv.scan("rowTimeT").select("addTime, id, name, val")
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        "StreamTableSourceScan(table=[[rowTimeT]], fields=[id, val, name, addTime])",
+        term("select", "addTime", "id", "name", "val")
+      )
+    util.verifyTable(t, expected)
+  }
+
+  @Test
+  def testRowTimeTableSourceGroupWindow(): Unit = {
+    val util = streamTestUtil()
+    util.tEnv.registerTableSource("rowTimeT", new TestRowTimeSource("addTime"))
+
+    val t = util.tEnv.scan("rowTimeT")
+      .filter("val > 100")
+      .window(Tumble over 10.minutes on 'addTime as 'w)
+      .groupBy('name, 'w)
+      .select('name, 'w.end, 'val.avg)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            "StreamTableSourceScan(table=[[rowTimeT]], fields=[id, val, name, addTime])",
+            term("select", "name", "val", "addTime"),
+            term("where", ">(val, 100)")
+          ),
+          term("groupBy", "name"),
+          term("window", "TumblingGroupWindow(WindowReference(w), 'addTime, 600000.millis)"),
+          term("select", "name", "AVG(val) AS TMP_1", "end(WindowReference(w)) AS TMP_0")
+        ),
+        term("select", "name", "TMP_0", "TMP_1")
+      )
+    util.verifyTable(t, expected)
+  }
+
+  @Test
+  def testProcTimeTableSourceSimple(): Unit = {
+    val util = streamTestUtil()
+    util.tEnv.registerTableSource("procTimeT", new TestProcTimeSource("pTime"))
+
+    val t = util.tEnv.scan("procTimeT").select("pTime, id, name, val")
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        "StreamTableSourceScan(table=[[procTimeT]], fields=[id, val, name, pTime])",
+        term("select", "pTime", "id", "name", "val")
+      )
+    util.verifyTable(t, expected)
+  }
+
+  @Test
+  def testProcTimeTableSourceOverWindow(): Unit = {
+    val util = streamTestUtil()
+    util.tEnv.registerTableSource("procTimeT", new TestProcTimeSource("pTime"))
+
+    val t = util.tEnv.scan("procTimeT")
+      .window(Over partitionBy 'id orderBy 'pTime preceding 2.hours as 'w)
+      .select('id, 'name, 'val.sum over 'w as 'valSum)
+      .filter('valSum > 100)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          "StreamTableSourceScan(table=[[procTimeT]], fields=[id, val, name, pTime])",
+          term("partitionBy", "id"),
+          term("orderBy", "pTime"),
+          term("range", "BETWEEN 7200000 PRECEDING AND CURRENT ROW"),
+          term("select", "id", "val", "name", "pTime", "SUM(val) AS w0$o0")
+        ),
+        term("select", "id", "name", "w0$o0 AS valSum"),
+        term("where", ">(w0$o0, 100)")
+      )
+    util.verifyTable(t, expected)
+  }
+}
+
+class TestRowTimeSource(timeField: String)
+    extends StreamTableSource[Row] with DefinedRowTimeAttribute {
+
+  override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = ???
+
+  override def getRowtimeAttribute: String = timeField
+
+  override def getReturnType: TypeInformation[Row] = {
+    new RowTypeInfo(
+      Array(Types.INT, Types.LONG, Types.STRING).asInstanceOf[Array[TypeInformation[_]]],
+      Array("id", "val", "name"))
+  }
+}
+
+class TestProcTimeSource(timeField: String)
+    extends StreamTableSource[Row] with DefinedProcTimeAttribute {
+
+  override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = ???
+
+  override def getProctimeAttribute: String = timeField
+
+  override def getReturnType: TypeInformation[Row] = {
+    new RowTypeInfo(
+      Array(Types.INT, Types.LONG, Types.STRING).asInstanceOf[Array[TypeInformation[_]]],
+      Array("id", "val", "name"))
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/24bf61ce/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index edf7b1d..f84ae3d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.utils.TableTestUtil._
 import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
-import org.junit.{Ignore, Test}
+import org.junit.Test
 
 class WindowAggregateTest extends TableTestBase {
   private val streamUtil: StreamTableTestUtil = streamTestUtil()
@@ -85,7 +85,6 @@ class WindowAggregateTest extends TableTestBase {
   }
 
   @Test
-  @Ignore // TODO enable once CALCITE-1761 is fixed
   def testTumbleFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
@@ -98,24 +97,23 @@ class WindowAggregateTest extends TableTestBase {
         "GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE)"
     val expected =
       unaryNode(
-        "DataStreamCalc",
+        "DataStreamAggregate",
         unaryNode(
-          "DataStreamAggregate",
+          "DataStreamCalc",
           streamTableNode(0),
-          term("window", TumblingGroupWindow('w$, 'rowtime, 900000.millis)),
-          term("select",
-            "COUNT(*) AS EXPR$0, " +
-              "weightedAvg(c, a) AS wAvg, " +
-              "start('w$) AS w$start, " +
-              "end('w$) AS w$end")
+          term("select", "1970-01-01 00:00:00 AS $f0", "c", "a")
         ),
-        term("select", "EXPR$0, wAvg, CAST(w$start) AS w$start, CAST(w$end) AS w$end")
+        term("window", TumblingGroupWindow('w$, 'rowtime, 900000.millis)),
+        term("select",
+          "COUNT(*) AS EXPR$0, " +
+            "weightedAvg(c, a) AS wAvg, " +
+            "start('w$) AS w$start, " +
+            "end('w$) AS w$end")
       )
     streamUtil.verifySql(sql, expected)
   }
 
   @Test
-  @Ignore // TODO enable once CALCITE-1761 is fixed
   def testHoppingFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
@@ -127,24 +125,23 @@ class WindowAggregateTest extends TableTestBase {
         "GROUP BY HOP(proctime, INTERVAL '15' MINUTE, INTERVAL '1' HOUR)"
     val expected =
       unaryNode(
-        "DataStreamCalc",
+        "DataStreamAggregate",
         unaryNode(
-          "DataStreamAggregate",
+          "DataStreamCalc",
           streamTableNode(0),
-          term("window", SlidingGroupWindow('w$, 'rowtime, 3600000.millis, 900000.millis)),
-          term("select",
-            "COUNT(*) AS EXPR$0, " +
-              "weightedAvg(c, a) AS wAvg, " +
-              "start('w$) AS w$start, " +
-              "end('w$) AS w$end")
+          term("select", "1970-01-01 00:00:00 AS $f0", "c", "a")
         ),
-        term("select", "EXPR$0, wAvg, CAST(w$start) AS w$start, CAST(w$end) AS w$end")
+        term("window", SlidingGroupWindow('w$, 'proctime, 3600000.millis, 900000.millis)),
+        term("select",
+          "COUNT(*) AS EXPR$0, " +
+            "weightedAvg(c, a) AS wAvg, " +
+            "start('w$) AS w$start, " +
+            "end('w$) AS w$end")
       )
     streamUtil.verifySql(sql, expected)
   }
 
   @Test
-  @Ignore // TODO enable once CALCITE-1761 is fixed
   def testSessionFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
@@ -157,18 +154,18 @@ class WindowAggregateTest extends TableTestBase {
         "GROUP BY SESSION(proctime, INTERVAL '15' MINUTE)"
     val expected =
       unaryNode(
-        "DataStreamCalc",
+        "DataStreamAggregate",
         unaryNode(
-          "DataStreamAggregate",
+          "DataStreamCalc",
           streamTableNode(0),
-          term("window", SessionGroupWindow('w$, 'rowtime, 900000.millis)),
-          term("select",
-            "COUNT(*) AS EXPR$0, " +
-              "weightedAvg(c, a) AS wAvg, " +
-              "start('w$) AS w$start, " +
-              "end('w$) AS w$end")
+          term("select", "1970-01-01 00:00:00 AS $f0", "c", "a")
         ),
-        term("select", "EXPR$0, wAvg, CAST(w$start) AS w$start, CAST(w$end) AS w$end")
+        term("window", SessionGroupWindow('w$, 'proctime, 900000.millis)),
+        term("select",
+          "COUNT(*) AS EXPR$0, " +
+            "weightedAvg(c, a) AS wAvg, " +
+            "start('w$) AS w$start, " +
+            "end('w$) AS w$end")
       )
     streamUtil.verifySql(sql, expected)
   }


[09/15] flink git commit: [FLINK-6216] [table] Add non-windowed GroupBy aggregation for streams.

Posted by fh...@apache.org.
[FLINK-6216] [table] Add non-windowed GroupBy aggregation for streams.

This closes #3646.


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

Branch: refs/heads/master
Commit: 8f78824be2e6b5e2029f142b70f7cac15d98abd3
Parents: 24bf61c
Author: shaoxuan-wang <ws...@gmail.com>
Authored: Thu Mar 30 03:57:58 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:54 2017 +0200

----------------------------------------------------------------------
 .../flink/table/plan/logical/operators.scala    |   3 -
 .../nodes/datastream/DataStreamAggregate.scala  | 300 -------------------
 .../datastream/DataStreamGroupAggregate.scala   | 152 ++++++++++
 .../DataStreamGroupWindowAggregate.scala        | 300 +++++++++++++++++++
 .../flink/table/plan/rules/FlinkRuleSets.scala  |   3 +-
 .../datastream/DataStreamAggregateRule.scala    |  77 -----
 .../DataStreamGroupAggregateRule.scala          |  81 +++++
 .../DataStreamGroupWindowAggregateRule.scala    |  77 +++++
 .../table/runtime/aggregate/AggregateUtil.scala |  66 +++-
 .../aggregate/GroupAggProcessFunction.scala     |  90 ++++++
 .../aggregate/ProcTimeBoundedRangeOver.scala    |   2 +-
 .../scala/batch/table/FieldProjectionTest.scala |   4 +-
 .../api/scala/stream/TableSourceTest.scala      |   2 +-
 .../table/api/scala/stream/sql/SqlITCase.scala  |  21 ++
 .../scala/stream/sql/WindowAggregateTest.scala  |  51 ++--
 .../scala/stream/table/AggregationsITCase.scala | 197 ------------
 .../stream/table/GroupAggregationsITCase.scala  | 132 ++++++++
 .../stream/table/GroupAggregationsTest.scala    | 214 +++++++++++++
 .../table/GroupWindowAggregationsITCase.scala   | 197 ++++++++++++
 .../scala/stream/table/GroupWindowTest.scala    | 195 +++++++-----
 .../scala/stream/table/UnsupportedOpsTest.scala |   7 -
 21 files changed, 1480 insertions(+), 691 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/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 3839145..36067eb 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
@@ -215,9 +215,6 @@ case class Aggregate(
   }
 
   override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Aggregate on stream tables is currently not supported.")
-    }
 
     val resolvedAggregate = super.validate(tableEnv).asInstanceOf[Aggregate]
     val groupingExprs = resolvedAggregate.groupingExpressions

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
deleted file mode 100644
index 5697449..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * 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.table.plan.nodes.datastream
-
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.AggregateCall
-import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream}
-import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
-import org.apache.flink.table.api.StreamTableEnvironment
-import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.table.codegen.CodeGenerator
-import org.apache.flink.table.expressions.ExpressionUtils._
-import org.apache.flink.table.plan.logical._
-import org.apache.flink.table.plan.nodes.CommonAggregate
-import org.apache.flink.table.plan.nodes.datastream.DataStreamAggregate._
-import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.aggregate.AggregateUtil._
-import org.apache.flink.table.runtime.aggregate._
-import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
-import org.apache.flink.types.Row
-
-class DataStreamAggregate(
-    window: LogicalWindow,
-    namedProperties: Seq[NamedWindowProperty],
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    inputNode: RelNode,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    schema: RowSchema,
-    inputSchema: RowSchema,
-    grouping: Array[Int])
-  extends SingleRel(cluster, traitSet, inputNode) with CommonAggregate with DataStreamRel {
-
-  override def deriveRowType(): RelDataType = schema.logicalType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamAggregate(
-      window,
-      namedProperties,
-      cluster,
-      traitSet,
-      inputs.get(0),
-      namedAggregates,
-      schema,
-      inputSchema,
-      grouping)
-  }
-
-  override def toString: String = {
-    s"Aggregate(${
-      if (!grouping.isEmpty) {
-        s"groupBy: (${groupingToString(inputSchema.logicalType, grouping)}), "
-      } else {
-        ""
-      }
-    }window: ($window), " +
-      s"select: (${
-        aggregationToString(
-          inputSchema.logicalType,
-          grouping,
-          getRowType,
-          namedAggregates,
-          namedProperties)
-      }))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .itemIf("groupBy", groupingToString(inputSchema.logicalType, grouping), !grouping.isEmpty)
-      .item("window", window)
-      .item(
-        "select", aggregationToString(
-          inputSchema.logicalType,
-          grouping,
-          schema.logicalType,
-          namedAggregates,
-          namedProperties))
-  }
-
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
-
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-    val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
-      new CalcitePair[AggregateCall, String](
-        inputSchema.mapAggregateCall(namedAggregate.left),
-        namedAggregate.right)
-    }
-
-    val aggString = aggregationToString(
-      inputSchema.logicalType,
-      grouping,
-      schema.logicalType,
-      namedAggregates,
-      namedProperties)
-
-    val keyedAggOpName = s"groupBy: (${groupingToString(schema.logicalType, grouping)}), " +
-      s"window: ($window), " +
-      s"select: ($aggString)"
-    val nonKeyedAggOpName = s"window: ($window), select: ($aggString)"
-
-    val generator = new CodeGenerator(
-      tableEnv.getConfig,
-      false,
-      inputDS.getType)
-
-    val needMerge = window match {
-      case SessionGroupWindow(_, _, _) => true
-      case _ => false
-    }
-    val physicalGrouping = grouping.map(inputSchema.mapIndex)
-
-    // grouped / keyed aggregation
-    if (physicalGrouping.length > 0) {
-      val windowFunction = AggregateUtil.createAggregationGroupWindowFunction(
-        window,
-        physicalGrouping.length,
-        physicalNamedAggregates.size,
-        schema.physicalArity,
-        namedProperties)
-
-      val keyedStream = inputDS.keyBy(physicalGrouping: _*)
-      val windowedStream =
-        createKeyedWindowedStream(window, keyedStream)
-          .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
-
-      val (aggFunction, accumulatorRowType, aggResultRowType) =
-        AggregateUtil.createDataStreamAggregateFunction(
-          generator,
-          physicalNamedAggregates,
-          inputSchema.physicalType,
-          inputSchema.physicalFieldTypeInfo,
-          schema.physicalType,
-          needMerge)
-
-      windowedStream
-        .aggregate(
-          aggFunction,
-          windowFunction,
-          accumulatorRowType,
-          aggResultRowType,
-          schema.physicalTypeInfo)
-        .name(keyedAggOpName)
-    }
-    // global / non-keyed aggregation
-    else {
-      val windowFunction = AggregateUtil.createAggregationAllWindowFunction(
-        window,
-        schema.physicalArity,
-        namedProperties)
-
-      val windowedStream =
-        createNonKeyedWindowedStream(window, inputDS)
-          .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
-
-      val (aggFunction, accumulatorRowType, aggResultRowType) =
-        AggregateUtil.createDataStreamAggregateFunction(
-          generator,
-          physicalNamedAggregates,
-          inputSchema.physicalType,
-          inputSchema.physicalFieldTypeInfo,
-          schema.physicalType,
-          needMerge)
-
-      windowedStream
-        .aggregate(
-          aggFunction,
-          windowFunction,
-          accumulatorRowType,
-          aggResultRowType,
-          schema.physicalTypeInfo)
-        .name(nonKeyedAggOpName)
-    }
-  }
-}
-
-object DataStreamAggregate {
-
-
-  private def createKeyedWindowedStream(groupWindow: LogicalWindow, stream: KeyedStream[Row, Tuple])
-    : WindowedStream[Row, Tuple, _ <: DataStreamWindow] = groupWindow match {
-
-    case TumblingGroupWindow(_, timeField, size)
-        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
-      stream.window(TumblingProcessingTimeWindows.of(toTime(size)))
-
-    case TumblingGroupWindow(_, timeField, size)
-        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
-      stream.countWindow(toLong(size))
-
-    case TumblingGroupWindow(_, timeField, size)
-        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
-      stream.window(TumblingEventTimeWindows.of(toTime(size)))
-
-    case TumblingGroupWindow(_, _, size) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException(
-        "Event-time grouping windows on row intervals are currently not supported.")
-
-    case SlidingGroupWindow(_, timeField, size, slide)
-        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(slide) =>
-      stream.window(SlidingProcessingTimeWindows.of(toTime(size), toTime(slide)))
-
-    case SlidingGroupWindow(_, timeField, size, slide)
-        if isProctimeAttribute(timeField) && isRowCountLiteral(size) =>
-      stream.countWindow(toLong(size), toLong(slide))
-
-    case SlidingGroupWindow(_, timeField, size, slide)
-        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
-      stream.window(SlidingEventTimeWindows.of(toTime(size), toTime(slide)))
-
-    case SlidingGroupWindow(_, _, size, slide) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException(
-        "Event-time grouping windows on row intervals are currently not supported.")
-
-    case SessionGroupWindow(_, timeField, gap)
-        if isProctimeAttribute(timeField) =>
-      stream.window(ProcessingTimeSessionWindows.withGap(toTime(gap)))
-
-    case SessionGroupWindow(_, timeField, gap)
-        if isRowtimeAttribute(timeField) =>
-      stream.window(EventTimeSessionWindows.withGap(toTime(gap)))
-  }
-
-  private def createNonKeyedWindowedStream(groupWindow: LogicalWindow, stream: DataStream[Row])
-    : AllWindowedStream[Row, _ <: DataStreamWindow] = groupWindow match {
-
-    case TumblingGroupWindow(_, timeField, size)
-        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
-      stream.windowAll(TumblingProcessingTimeWindows.of(toTime(size)))
-
-    case TumblingGroupWindow(_, timeField, size)
-        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
-      stream.countWindowAll(toLong(size))
-
-    case TumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
-      stream.windowAll(TumblingEventTimeWindows.of(toTime(size)))
-
-    case TumblingGroupWindow(_, _, size) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException(
-        "Event-time grouping windows on row intervals are currently not supported.")
-
-    case SlidingGroupWindow(_, timeField, size, slide)
-        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
-      stream.windowAll(SlidingProcessingTimeWindows.of(toTime(size), toTime(slide)))
-
-    case SlidingGroupWindow(_, timeField, size, slide)
-        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
-      stream.countWindowAll(toLong(size), toLong(slide))
-
-    case SlidingGroupWindow(_, timeField, size, slide)
-        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
-      stream.windowAll(SlidingEventTimeWindows.of(toTime(size), toTime(slide)))
-
-    case SlidingGroupWindow(_, _, size, slide) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException(
-        "Event-time grouping windows on row intervals are currently not supported.")
-
-    case SessionGroupWindow(_, timeField, gap)
-        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(gap) =>
-      stream.windowAll(ProcessingTimeSessionWindows.withGap(toTime(gap)))
-
-    case SessionGroupWindow(_, timeField, gap)
-        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(gap) =>
-      stream.windowAll(EventTimeSessionWindows.withGap(toTime(gap)))
-  }
-
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
new file mode 100644
index 0000000..19f90c7
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
@@ -0,0 +1,152 @@
+/*
+ * 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.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
+
+/**
+  *
+  * Flink RelNode for data stream unbounded group aggregate
+  *
+  * @param cluster         Cluster of the RelNode, represent for an environment of related
+  *                        relational expressions during the optimization of a query.
+  * @param traitSet        Trait set of the RelNode
+  * @param inputNode       The input RelNode of aggregation
+  * @param namedAggregates List of calls to aggregate functions and their output field names
+  * @param rowRelDataType  The type of the rows of the RelNode
+  * @param inputSchema     The type of the rows consumed by this RelNode
+  * @param schema          The type of the rows emitted by this RelNode
+  * @param groupings       The position (in the input Row) of the grouping keys
+  */
+class DataStreamGroupAggregate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputNode: RelNode,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    rowRelDataType: RelDataType,
+    schema: RowSchema,
+    inputSchema: RowSchema,
+    groupings: Array[Int])
+  extends SingleRel(cluster, traitSet, inputNode)
+    with CommonAggregate
+    with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamGroupAggregate(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      namedAggregates,
+      getRowType,
+      schema,
+      inputSchema,
+      groupings)
+  }
+
+  override def toString: String = {
+    s"Aggregate(${
+      if (!groupings.isEmpty) {
+        s"groupBy: (${groupingToString(inputSchema.logicalType, groupings)}), "
+      } else {
+        ""
+      }
+    }select:(${aggregationToString(
+      inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil)}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .itemIf("groupBy", groupingToString(
+        inputSchema.logicalType, groupings), !groupings.isEmpty)
+      .item("select", aggregationToString(
+        inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil))
+  }
+
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
+      new CalcitePair[AggregateCall, String](
+        inputSchema.mapAggregateCall(namedAggregate.left),
+        namedAggregate.right)
+    }
+
+    val generator = new CodeGenerator(
+      tableEnv.getConfig,
+      false,
+      inputDS.getType)
+
+    val aggString = aggregationToString(
+      inputSchema.logicalType,
+      groupings,
+      getRowType,
+      namedAggregates,
+      Nil)
+
+    val keyedAggOpName = s"groupBy: (${groupingToString(inputSchema.logicalType, groupings)}), " +
+      s"select: ($aggString)"
+    val nonKeyedAggOpName = s"select: ($aggString)"
+
+    val physicalGrouping = groupings.map(inputSchema.mapIndex)
+
+    val processFunction = AggregateUtil.createGroupAggregateFunction(
+      generator,
+      physicalNamedAggregates,
+      inputSchema.logicalType,
+      inputSchema.physicalFieldTypeInfo,
+      groupings)
+
+    val result: DataStream[Row] =
+    // grouped / keyed aggregation
+      if (physicalGrouping.nonEmpty) {
+        inputDS
+        .keyBy(groupings: _*)
+        .process(processFunction)
+        .returns(schema.physicalTypeInfo)
+        .name(keyedAggOpName)
+        .asInstanceOf[DataStream[Row]]
+      }
+      // global / non-keyed aggregation
+      else {
+        inputDS
+        .keyBy(new NullByteKeySelector[Row])
+        .process(processFunction)
+        .setParallelism(1)
+        .setMaxParallelism(1)
+        .returns(schema.physicalTypeInfo)
+        .name(nonKeyedAggOpName)
+        .asInstanceOf[DataStream[Row]]
+      }
+    result
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
new file mode 100644
index 0000000..5aced66
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
@@ -0,0 +1,300 @@
+/*
+ * 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.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream}
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
+import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.expressions.ExpressionUtils._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.plan.nodes.datastream.DataStreamGroupWindowAggregate._
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.types.Row
+
+class DataStreamGroupWindowAggregate(
+    window: LogicalWindow,
+    namedProperties: Seq[NamedWindowProperty],
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputNode: RelNode,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    schema: RowSchema,
+    inputSchema: RowSchema,
+    grouping: Array[Int])
+  extends SingleRel(cluster, traitSet, inputNode) with CommonAggregate with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamGroupWindowAggregate(
+      window,
+      namedProperties,
+      cluster,
+      traitSet,
+      inputs.get(0),
+      namedAggregates,
+      schema,
+      inputSchema,
+      grouping)
+  }
+
+  override def toString: String = {
+    s"Aggregate(${
+      if (!grouping.isEmpty) {
+        s"groupBy: (${groupingToString(inputSchema.logicalType, grouping)}), "
+      } else {
+        ""
+      }
+    }window: ($window), " +
+      s"select: (${
+        aggregationToString(
+          inputSchema.logicalType,
+          grouping,
+          getRowType,
+          namedAggregates,
+          namedProperties)
+      }))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .itemIf("groupBy", groupingToString(inputSchema.logicalType, grouping), !grouping.isEmpty)
+      .item("window", window)
+      .item(
+        "select", aggregationToString(
+          inputSchema.logicalType,
+          grouping,
+          schema.logicalType,
+          namedAggregates,
+          namedProperties))
+  }
+
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
+      new CalcitePair[AggregateCall, String](
+        inputSchema.mapAggregateCall(namedAggregate.left),
+        namedAggregate.right)
+    }
+
+    val aggString = aggregationToString(
+      inputSchema.logicalType,
+      grouping,
+      schema.logicalType,
+      namedAggregates,
+      namedProperties)
+
+    val keyedAggOpName = s"groupBy: (${groupingToString(schema.logicalType, grouping)}), " +
+      s"window: ($window), " +
+      s"select: ($aggString)"
+    val nonKeyedAggOpName = s"window: ($window), select: ($aggString)"
+
+    val generator = new CodeGenerator(
+      tableEnv.getConfig,
+      false,
+      inputDS.getType)
+
+    val needMerge = window match {
+      case SessionGroupWindow(_, _, _) => true
+      case _ => false
+    }
+    val physicalGrouping = grouping.map(inputSchema.mapIndex)
+
+    // grouped / keyed aggregation
+    if (physicalGrouping.length > 0) {
+      val windowFunction = AggregateUtil.createAggregationGroupWindowFunction(
+        window,
+        physicalGrouping.length,
+        physicalNamedAggregates.size,
+        schema.physicalArity,
+        namedProperties)
+
+      val keyedStream = inputDS.keyBy(physicalGrouping: _*)
+      val windowedStream =
+        createKeyedWindowedStream(window, keyedStream)
+          .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
+
+      val (aggFunction, accumulatorRowType, aggResultRowType) =
+        AggregateUtil.createDataStreamAggregateFunction(
+          generator,
+          physicalNamedAggregates,
+          inputSchema.physicalType,
+          inputSchema.physicalFieldTypeInfo,
+          schema.physicalType,
+          needMerge)
+
+      windowedStream
+        .aggregate(
+          aggFunction,
+          windowFunction,
+          accumulatorRowType,
+          aggResultRowType,
+          schema.physicalTypeInfo)
+        .name(keyedAggOpName)
+    }
+    // global / non-keyed aggregation
+    else {
+      val windowFunction = AggregateUtil.createAggregationAllWindowFunction(
+        window,
+        schema.physicalArity,
+        namedProperties)
+
+      val windowedStream =
+        createNonKeyedWindowedStream(window, inputDS)
+          .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
+
+      val (aggFunction, accumulatorRowType, aggResultRowType) =
+        AggregateUtil.createDataStreamAggregateFunction(
+          generator,
+          physicalNamedAggregates,
+          inputSchema.physicalType,
+          inputSchema.physicalFieldTypeInfo,
+          schema.physicalType,
+          needMerge)
+
+      windowedStream
+        .aggregate(
+          aggFunction,
+          windowFunction,
+          accumulatorRowType,
+          aggResultRowType,
+          schema.physicalTypeInfo)
+        .name(nonKeyedAggOpName)
+    }
+  }
+}
+
+object DataStreamGroupWindowAggregate {
+
+
+  private def createKeyedWindowedStream(groupWindow: LogicalWindow, stream: KeyedStream[Row, Tuple])
+    : WindowedStream[Row, Tuple, _ <: DataStreamWindow] = groupWindow match {
+
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
+      stream.window(TumblingProcessingTimeWindows.of(toTime(size)))
+
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
+      stream.countWindow(toLong(size))
+
+    case TumblingGroupWindow(_, timeField, size)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
+      stream.window(TumblingEventTimeWindows.of(toTime(size)))
+
+    case TumblingGroupWindow(_, _, size) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException(
+        "Event-time grouping windows on row intervals are currently not supported.")
+
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(slide) =>
+      stream.window(SlidingProcessingTimeWindows.of(toTime(size), toTime(slide)))
+
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size) =>
+      stream.countWindow(toLong(size), toLong(slide))
+
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
+      stream.window(SlidingEventTimeWindows.of(toTime(size), toTime(slide)))
+
+    case SlidingGroupWindow(_, _, size, slide) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException(
+        "Event-time grouping windows on row intervals are currently not supported.")
+
+    case SessionGroupWindow(_, timeField, gap)
+        if isProctimeAttribute(timeField) =>
+      stream.window(ProcessingTimeSessionWindows.withGap(toTime(gap)))
+
+    case SessionGroupWindow(_, timeField, gap)
+        if isRowtimeAttribute(timeField) =>
+      stream.window(EventTimeSessionWindows.withGap(toTime(gap)))
+  }
+
+  private def createNonKeyedWindowedStream(groupWindow: LogicalWindow, stream: DataStream[Row])
+    : AllWindowedStream[Row, _ <: DataStreamWindow] = groupWindow match {
+
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
+      stream.windowAll(TumblingProcessingTimeWindows.of(toTime(size)))
+
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
+      stream.countWindowAll(toLong(size))
+
+    case TumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
+      stream.windowAll(TumblingEventTimeWindows.of(toTime(size)))
+
+    case TumblingGroupWindow(_, _, size) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException(
+        "Event-time grouping windows on row intervals are currently not supported.")
+
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
+      stream.windowAll(SlidingProcessingTimeWindows.of(toTime(size), toTime(slide)))
+
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
+      stream.countWindowAll(toLong(size), toLong(slide))
+
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
+      stream.windowAll(SlidingEventTimeWindows.of(toTime(size), toTime(slide)))
+
+    case SlidingGroupWindow(_, _, size, slide) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException(
+        "Event-time grouping windows on row intervals are currently not supported.")
+
+    case SessionGroupWindow(_, timeField, gap)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(gap) =>
+      stream.windowAll(ProcessingTimeSessionWindows.withGap(toTime(gap)))
+
+    case SessionGroupWindow(_, timeField, gap)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(gap) =>
+      stream.windowAll(EventTimeSessionWindows.withGap(toTime(gap)))
+  }
+
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
index 838cb22..f4de651 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
@@ -176,8 +176,9 @@ object FlinkRuleSets {
     */
   val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList(
     // translate to DataStream nodes
+    DataStreamGroupAggregateRule.INSTANCE,
     DataStreamOverAggregateRule.INSTANCE,
-    DataStreamAggregateRule.INSTANCE,
+    DataStreamGroupWindowAggregateRule.INSTANCE,
     DataStreamCalcRule.INSTANCE,
     DataStreamScanRule.INSTANCE,
     DataStreamUnionRule.INSTANCE,

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
deleted file mode 100644
index fc65403..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.table.plan.rules.datastream
-
-import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.flink.table.api.TableException
-import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamAggregate
-import org.apache.flink.table.plan.nodes.logical.FlinkLogicalWindowAggregate
-import org.apache.flink.table.plan.schema.RowSchema
-
-import scala.collection.JavaConversions._
-
-class DataStreamAggregateRule
-  extends ConverterRule(
-    classOf[FlinkLogicalWindowAggregate],
-    FlinkConventions.LOGICAL,
-    FlinkConventions.DATASTREAM,
-    "DataStreamAggregateRule") {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val agg: FlinkLogicalWindowAggregate = call.rel(0).asInstanceOf[FlinkLogicalWindowAggregate]
-
-    // check if we have distinct aggregates
-    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
-    if (distinctAggs) {
-      throw TableException("DISTINCT aggregates are currently not supported.")
-    }
-
-    // check if we have grouping sets
-    val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
-    if (groupSets || agg.indicator) {
-      throw TableException("GROUPING SETS are currently not supported.")
-    }
-
-    !distinctAggs && !groupSets && !agg.indicator
-  }
-
-  override def convert(rel: RelNode): RelNode = {
-    val agg: FlinkLogicalWindowAggregate = rel.asInstanceOf[FlinkLogicalWindowAggregate]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
-    val convInput: RelNode = RelOptRule.convert(agg.getInput, FlinkConventions.DATASTREAM)
-
-    new DataStreamAggregate(
-      agg.getWindow,
-      agg.getNamedProperties,
-      rel.getCluster,
-      traitSet,
-      convInput,
-      agg.getNamedAggCalls,
-      new RowSchema(rel.getRowType),
-      new RowSchema(agg.getInput.getRowType),
-      agg.getGroupSet.toArray)
-    }
-  }
-
-object DataStreamAggregateRule {
-  val INSTANCE: RelOptRule = new DataStreamAggregateRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala
new file mode 100644
index 0000000..fd7619c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupAggregateRule.scala
@@ -0,0 +1,81 @@
+/*
+ * 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.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalAggregate
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.datastream.DataStreamGroupAggregate
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalAggregate
+import org.apache.flink.table.plan.schema.RowSchema
+
+import scala.collection.JavaConversions._
+
+/**
+  * Rule to convert a [[LogicalAggregate]] into a [[DataStreamGroupAggregate]].
+  */
+class DataStreamGroupAggregateRule
+  extends ConverterRule(
+    classOf[FlinkLogicalAggregate],
+    FlinkConventions.LOGICAL,
+    FlinkConventions.DATASTREAM,
+    "DataStreamGroupAggregateRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val agg: FlinkLogicalAggregate = call.rel(0).asInstanceOf[FlinkLogicalAggregate]
+
+    // check if we have distinct aggregates
+    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
+    if (distinctAggs) {
+      throw TableException("DISTINCT aggregates are currently not supported.")
+    }
+
+    // check if we have grouping sets
+    val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
+    if (groupSets || agg.indicator) {
+      throw TableException("GROUPING SETS are currently not supported.")
+    }
+
+    !distinctAggs && !groupSets && !agg.indicator
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val agg: FlinkLogicalAggregate = rel.asInstanceOf[FlinkLogicalAggregate]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
+    val convInput: RelNode = RelOptRule.convert(agg.getInput, FlinkConventions.DATASTREAM)
+
+    new DataStreamGroupAggregate(
+      rel.getCluster,
+      traitSet,
+      convInput,
+      agg.getNamedAggCalls,
+      rel.getRowType,
+      new RowSchema(rel.getRowType),
+      new RowSchema(agg.getInput.getRowType),
+      agg.getGroupSet.toArray)
+  }
+}
+
+object DataStreamGroupAggregateRule {
+  val INSTANCE: RelOptRule = new DataStreamGroupAggregateRule
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupWindowAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupWindowAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupWindowAggregateRule.scala
new file mode 100644
index 0000000..3beeb47
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamGroupWindowAggregateRule.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.datastream.DataStreamGroupWindowAggregate
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalWindowAggregate
+import org.apache.flink.table.plan.schema.RowSchema
+
+import scala.collection.JavaConversions._
+
+class DataStreamGroupWindowAggregateRule
+  extends ConverterRule(
+    classOf[FlinkLogicalWindowAggregate],
+    FlinkConventions.LOGICAL,
+    FlinkConventions.DATASTREAM,
+    "DataStreamGroupWindowAggregateRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val agg: FlinkLogicalWindowAggregate = call.rel(0).asInstanceOf[FlinkLogicalWindowAggregate]
+
+    // check if we have distinct aggregates
+    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
+    if (distinctAggs) {
+      throw TableException("DISTINCT aggregates are currently not supported.")
+    }
+
+    // check if we have grouping sets
+    val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
+    if (groupSets || agg.indicator) {
+      throw TableException("GROUPING SETS are currently not supported.")
+    }
+
+    !distinctAggs && !groupSets && !agg.indicator
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val agg: FlinkLogicalWindowAggregate = rel.asInstanceOf[FlinkLogicalWindowAggregate]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
+    val convInput: RelNode = RelOptRule.convert(agg.getInput, FlinkConventions.DATASTREAM)
+
+    new DataStreamGroupWindowAggregate(
+      agg.getWindow,
+      agg.getNamedProperties,
+      rel.getCluster,
+      traitSet,
+      convInput,
+      agg.getNamedAggCalls,
+      new RowSchema(rel.getRowType),
+      new RowSchema(agg.getInput.getRowType),
+      agg.getGroupSet.toArray)
+    }
+  }
+
+object DataStreamGroupWindowAggregateRule {
+  val INSTANCE: RelOptRule = new DataStreamGroupWindowAggregateRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index dfed34a..5e9efd0 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -87,8 +87,7 @@ object AggregateUtil {
         inputType,
         needRetraction = false)
 
-    val aggregationStateType: RowTypeInfo =
-      createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
+    val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
 
     val forwardMapping = (0 until inputType.getFieldCount).toArray
     val aggMapping = aggregates.indices.map(x => x + inputType.getFieldCount).toArray
@@ -139,7 +138,58 @@ object AggregateUtil {
   }
 
   /**
-    * Create an [[org.apache.flink.streaming.api.functions.ProcessFunction]] for
+    * Create an [[org.apache.flink.streaming.api.functions.ProcessFunction]] for group (without
+    * window) aggregate to evaluate final aggregate value.
+    *
+    * @param generator       code generator instance
+    * @param namedAggregates List of calls to aggregate functions and their output field names
+    * @param inputRowType    Input row type
+    * @param inputFieldTypes Types of the physical input fields
+    * @param groupings       the position (in the input Row) of the grouping keys
+    * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
+    */
+  private[flink] def createGroupAggregateFunction(
+      generator: CodeGenerator,
+      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+      inputRowType: RelDataType,
+      inputFieldTypes: Seq[TypeInformation[_]],
+      groupings: Array[Int]): ProcessFunction[Row, Row] = {
+
+    val (aggFields, aggregates) =
+      transformToAggregateFunctions(
+        namedAggregates.map(_.getKey),
+        inputRowType,
+        needRetraction = false)
+    val aggMapping = aggregates.indices.map(_ + groupings.length).toArray
+
+    val outputArity = groupings.length + aggregates.length
+
+    val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
+
+    val genFunction = generator.generateAggregations(
+      "NonWindowedAggregationHelper",
+      generator,
+      inputFieldTypes,
+      aggregates,
+      aggFields,
+      aggMapping,
+      partialResults = false,
+      groupings,
+      None,
+      None,
+      outputArity,
+      needRetract = false,
+      needMerge = false,
+      needReset = false
+    )
+
+    new GroupAggProcessFunction(
+      genFunction,
+      aggregationStateType)
+  }
+
+  /**
+    * Create an [[org.apache.flink.streaming.api.functions.ProcessFunction]] for ROWS clause
     * bounded OVER window to evaluate final aggregate value.
     *
     * @param generator       code generator instance
@@ -266,7 +316,7 @@ object AggregateUtil {
       needRetract)
 
     val mapReturnType: RowTypeInfo =
-      createDataSetAggregateBufferDataType(
+      createRowTypeForKeysAndAggregates(
         groupings,
         aggregates,
         inputType,
@@ -370,7 +420,7 @@ object AggregateUtil {
       physicalInputRowType,
       needRetract)
 
-    val returnType: RowTypeInfo = createDataSetAggregateBufferDataType(
+    val returnType: RowTypeInfo = createRowTypeForKeysAndAggregates(
       groupings,
       aggregates,
       physicalInputRowType,
@@ -637,7 +687,7 @@ object AggregateUtil {
     window match {
       case SessionGroupWindow(_, _, gap) =>
         val combineReturnType: RowTypeInfo =
-          createDataSetAggregateBufferDataType(
+          createRowTypeForKeysAndAggregates(
             groupings,
             aggregates,
             physicalInputRowType,
@@ -711,7 +761,7 @@ object AggregateUtil {
 
       case SessionGroupWindow(_, _, gap) =>
         val combineReturnType: RowTypeInfo =
-          createDataSetAggregateBufferDataType(
+          createRowTypeForKeysAndAggregates(
             groupings,
             aggregates,
             physicalInputRowType,
@@ -1365,7 +1415,7 @@ object AggregateUtil {
     aggTypes
   }
 
-  private def createDataSetAggregateBufferDataType(
+  private def createRowTypeForKeysAndAggregates(
       groupings: Array[Int],
       aggregates: Array[TableAggregateFunction[_, _]],
       inputType: RelDataType,

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
new file mode 100644
index 0000000..81c900c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
@@ -0,0 +1,90 @@
+/*
+ * 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.table.runtime.aggregate
+
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.slf4j.LoggerFactory
+
+/**
+  * Aggregate Function used for the groupby (without window) aggregate
+  *
+  * @param genAggregations      Generated aggregate helper function
+  * @param aggregationStateType The row type info of aggregation
+  */
+class GroupAggProcessFunction(
+    private val genAggregations: GeneratedAggregationsFunction,
+    private val aggregationStateType: RowTypeInfo)
+  extends ProcessFunction[Row, Row]
+    with Compiler[GeneratedAggregations] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  private var function: GeneratedAggregations = _
+
+  private var output: Row = _
+  private var state: ValueState[Row] = _
+
+  override def open(config: Configuration) {
+    LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " +
+      s"Code:\n$genAggregations.code")
+    val clazz = compile(
+      getRuntimeContext.getUserCodeClassLoader,
+      genAggregations.name,
+      genAggregations.code)
+    LOG.debug("Instantiating AggregateHelper.")
+    function = clazz.newInstance()
+    output = function.createOutputRow()
+
+    val stateDescriptor: ValueStateDescriptor[Row] =
+      new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType)
+    state = getRuntimeContext.getState(stateDescriptor)
+  }
+
+  override def processElement(
+      input: Row,
+      ctx: ProcessFunction[Row, Row]#Context,
+      out: Collector[Row]): Unit = {
+
+    // get accumulators
+    var accumulators = state.value()
+    if (null == accumulators) {
+      accumulators = function.createAccumulators()
+    }
+
+    // Set group keys value to the final output
+    function.setForwardedFields(input, output)
+
+    // accumulate new input row
+    function.accumulate(accumulators, input)
+
+    // set aggregation results to output
+    function.setAggregationResults(accumulators, output)
+
+    // update accumulators
+    state.update(accumulators)
+
+    out.collect(output)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
index 7f87e50..b63eb81 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory
   * Process Function used for the aggregate in bounded proc-time OVER window
   * [[org.apache.flink.streaming.api.datastream.DataStream]]
   *
-  * @param genAggregations Generated aggregate helper function
+  * @param genAggregations          Generated aggregate helper function
   * @param precedingTimeBoundary    Is used to indicate the processing time boundaries
   * @param aggregatesTypeInfo       row type info of aggregation
   * @param inputType                row type info of input row

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
index b484293..93e25f8 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
@@ -230,7 +230,7 @@ class FieldProjectionTest extends TableTestBase {
 
     val expected =
       unaryNode(
-        "DataStreamAggregate",
+        "DataStreamGroupWindowAggregate",
         unaryNode(
           "DataStreamCalc",
           streamTableNode(0),
@@ -259,7 +259,7 @@ class FieldProjectionTest extends TableTestBase {
     val expected = unaryNode(
         "DataStreamCalc",
         unaryNode(
-          "DataStreamAggregate",
+          "DataStreamGroupWindowAggregate",
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
index 7673266..18066c9 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
@@ -63,7 +63,7 @@ class TableSourceTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         unaryNode(
-          "DataStreamAggregate",
+          "DataStreamGroupWindowAggregate",
           unaryNode(
             "DataStreamCalc",
             "StreamTableSourceScan(table=[[rowTimeT]], fields=[id, val, name, addTime])",

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index 6bab4b3..abbcbdd 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -47,6 +47,27 @@ class SqlITCase extends StreamingWithStateTestBase {
     (8L, 8, "Hello World"),
     (20L, 20, "Hello World"))
 
+  /** test unbounded groupby (without window) **/
+  @Test
+  def testUnboundedGroupby(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT b, COUNT(a) FROM MyTable GROUP BY b"
+
+    val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", t)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("1,1", "2,1", "2,2")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
   /** test selection **/
   @Test
   def testSelectExpressionFromTable(): Unit = {

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index f84ae3d..4c1d6e6 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -60,10 +60,13 @@ class WindowAggregateTest extends TableTestBase {
   @Test
   def testPartitionedProcessingTimeBoundedWindow() = {
 
-    val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY proctime " +
-      "RANGE BETWEEN INTERVAL '2' HOUR PRECEDING AND CURRENT ROW) AS avgA " +
+    val sqlQuery =
+      "SELECT a, " +
+      "  AVG(c) OVER (PARTITION BY a ORDER BY proctime " +
+      "    RANGE BETWEEN INTERVAL '2' HOUR PRECEDING AND CURRENT ROW) AS avgA " +
       "FROM MyTable"
-      val expected =
+
+    val expected =
       unaryNode(
         "DataStreamCalc",
         unaryNode(
@@ -85,6 +88,27 @@ class WindowAggregateTest extends TableTestBase {
   }
 
   @Test
+  def testGroupbyWithoutWindow() = {
+    val sql = "SELECT COUNT(a) FROM MyTable GROUP BY b"
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "b", "a")
+          ),
+          term("groupBy", "b"),
+          term("select", "b", "COUNT(a) AS EXPR$0")
+        ),
+        term("select", "EXPR$0")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
+  @Test
   def testTumbleFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
@@ -97,7 +121,7 @@ class WindowAggregateTest extends TableTestBase {
         "GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE)"
     val expected =
       unaryNode(
-        "DataStreamAggregate",
+        "DataStreamGroupWindowAggregate",
         unaryNode(
           "DataStreamCalc",
           streamTableNode(0),
@@ -125,7 +149,7 @@ class WindowAggregateTest extends TableTestBase {
         "GROUP BY HOP(proctime, INTERVAL '15' MINUTE, INTERVAL '1' HOUR)"
     val expected =
       unaryNode(
-        "DataStreamAggregate",
+        "DataStreamGroupWindowAggregate",
         unaryNode(
           "DataStreamCalc",
           streamTableNode(0),
@@ -154,7 +178,7 @@ class WindowAggregateTest extends TableTestBase {
         "GROUP BY SESSION(proctime, INTERVAL '15' MINUTE)"
     val expected =
       unaryNode(
-        "DataStreamAggregate",
+        "DataStreamGroupWindowAggregate",
         unaryNode(
           "DataStreamCalc",
           streamTableNode(0),
@@ -206,21 +230,6 @@ class WindowAggregateTest extends TableTestBase {
     streamUtil.verifySql(sql, "n/a")
   }
 
-  @Test(expected = classOf[TableException])
-  def testMultiWindow() = {
-    val sql = "SELECT COUNT(*) FROM MyTable GROUP BY " +
-      "FLOOR(rowtime TO HOUR), FLOOR(rowtime TO MINUTE)"
-    val expected = ""
-    streamUtil.verifySql(sql, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testInvalidWindowExpression() = {
-    val sql = "SELECT COUNT(*) FROM MyTable GROUP BY FLOOR(localTimestamp TO HOUR)"
-    val expected = ""
-    streamUtil.verifySql(sql, expected)
-  }
-
   @Test(expected = classOf[ValidationException])
   def testWindowUdAggInvalidArgs(): Unit = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
deleted file mode 100644
index 4a6a616..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * 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.table.api.scala.stream.table
-
-import org.apache.flink.api.scala._
-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.apache.flink.table.api.TableEnvironment
-import org.apache.flink.table.api.java.utils.UserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge}
-import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.scala.stream.table.AggregationsITCase.TimestampAndWatermarkWithOffset
-import org.apache.flink.table.api.scala.stream.utils.StreamITCase
-import org.apache.flink.table.functions.aggfunctions.CountAggFunction
-import org.apache.flink.types.Row
-import org.junit.Assert._
-import org.junit.Test
-
-import scala.collection.mutable
-
-/**
-  * We only test some aggregations until better testing of constructed DataStream
-  * programs is possible.
-  */
-class AggregationsITCase extends StreamingMultipleProgramsTestBase {
-
-  val data = List(
-    (1L, 1, "Hi"),
-    (2L, 2, "Hello"),
-    (4L, 2, "Hello"),
-    (8L, 3, "Hello world"),
-    (16L, 3, "Hello world"))
-
-  @Test
-  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setParallelism(1)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime)
-
-    val countFun = new CountAggFunction
-    val weightAvgFun = new WeightedAvg
-
-    val windowedTable = table
-      .window(Slide over 2.rows every 1.rows on 'proctime as 'w)
-      .groupBy('w, 'string)
-      .select('string, countFun('int), 'int.avg,
-              weightAvgFun('long, 'int), weightAvgFun('int, 'int))
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq("Hello world,1,3,8,3", "Hello world,2,3,12,3", "Hello,1,2,2,2",
-                       "Hello,2,2,3,2", "Hi,1,1,1,1")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testEventTimeSessionGroupWindowOverTime(): Unit = {
-    //To verify the "merge" functionality, we create this test with the following characteristics:
-    // 1. set the Parallelism to 1, and have the test data out of order
-    // 2. create a waterMark with 10ms offset to delay the window emission by 10ms
-    val sessionWindowTestdata = List(
-      (1L, 1, "Hello"),
-      (2L, 2, "Hello"),
-      (8L, 8, "Hello"),
-      (9L, 9, "Hello World"),
-      (4L, 4, "Hello"),
-      (16L, 16, "Hello"))
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val countFun = new CountAggFunction
-    val weightAvgFun = new WeightedAvgWithMerge
-
-    val stream = env
-      .fromCollection(sessionWindowTestdata)
-      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset(10L))
-    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime)
-
-    val windowedTable = table
-      .window(Session withGap 5.milli on 'rowtime as 'w)
-      .groupBy('w, 'string)
-      .select('string, countFun('int), 'int.avg,
-              weightAvgFun('long, 'int), weightAvgFun('int, 'int))
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq("Hello World,1,9,9,9", "Hello,1,16,16,16", "Hello,4,3,5,5")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setParallelism(1)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime)
-    val countFun = new CountAggFunction
-    val weightAvgFun = new WeightedAvg
-
-    val windowedTable = table
-      .window(Tumble over 2.rows on 'proctime as 'w)
-      .groupBy('w)
-      .select(countFun('string), 'int.avg,
-              weightAvgFun('long, 'int), weightAvgFun('int, 'int))
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq("2,1,1,1", "2,2,6,2")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testEventTimeTumblingWindow(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val stream = env
-      .fromCollection(data)
-      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset(0L))
-    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime)
-    val countFun = new CountAggFunction
-    val weightAvgFun = new WeightedAvg
-
-    val windowedTable = table
-      .window(Tumble over 5.milli on 'rowtime as 'w)
-      .groupBy('w, 'string)
-      .select('string, countFun('string), 'int.avg, weightAvgFun('long, 'int),
-              weightAvgFun('int, 'int), 'int.min, 'int.max, 'int.sum, 'w.start, 'w.end)
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq(
-      "Hello world,1,3,8,3,3,3,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01",
-      "Hello world,1,3,16,3,3,3,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02",
-      "Hello,2,2,3,2,2,2,4,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005",
-      "Hi,1,1,1,1,1,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-}
-
-object AggregationsITCase {
-  class TimestampAndWatermarkWithOffset(
-    offset: Long) extends AssignerWithPunctuatedWatermarks[(Long, Int, String)] {
-
-    override def checkAndGetNextWatermark(
-        lastElement: (Long, Int, String),
-        extractedTimestamp: Long)
-      : Watermark = {
-      new Watermark(extractedTimestamp - offset)
-    }
-
-    override def extractTimestamp(
-        element: (Long, Int, String),
-        previousElementTimestamp: Long): Long = {
-      element._1
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
new file mode 100644
index 0000000..271e90b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
@@ -0,0 +1,132 @@
+/*
+ * 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.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase}
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.types.Row
+import org.junit.Assert.assertEquals
+import org.junit.Test
+
+import scala.collection.mutable
+
+/**
+  * Tests of groupby (without window) aggregations
+  */
+class GroupAggregationsITCase extends StreamingWithStateTestBase {
+
+  @Test
+  def testNonKeyedGroupAggregate(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+            .select('a.sum, 'b.sum)
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,1", "3,3", "6,5", "10,8", "15,11", "21,14", "28,18", "36,22", "45,26", "55,30", "66,35",
+      "78,40", "91,45", "105,50", "120,55", "136,61", "153,67", "171,73", "190,79", "210,85",
+      "231,91")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testGroupAggregate(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('b)
+      .select('b, 'a.sum)
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,1", "2,2", "2,5", "3,4", "3,9", "3,15", "4,7", "4,15",
+      "4,24", "4,34", "5,11", "5,23", "5,36", "5,50", "5,65", "6,16", "6,33", "6,51", "6,70",
+      "6,90", "6,111")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testDoubleGroupAggregation(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('b)
+      .select('a.sum as 'd, 'b)
+      .groupBy('b, 'd)
+      .select('b)
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1",
+      "2", "2",
+      "3", "3", "3",
+      "4", "4", "4", "4",
+      "5", "5", "5", "5", "5",
+      "6", "6", "6", "6", "6", "6")
+
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testGroupAggregateWithExpression(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+      .groupBy('e, 'b % 3)
+      .select('c.min, 'e, 'a.avg, 'd.count)
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "0,1,1,1", "1,2,2,1", "2,1,2,1", "3,2,3,1", "1,2,2,2",
+      "5,3,3,1", "3,2,3,2", "7,1,4,1", "2,1,3,2", "3,2,3,3", "7,1,4,2", "5,3,4,2", "12,3,5,1",
+      "1,2,3,3", "14,2,5,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsTest.scala
new file mode 100644
index 0000000..520592c
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsTest.scala
@@ -0,0 +1,214 @@
+/*
+ * 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.table.api.scala.stream.table
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.utils.TableTestBase
+import org.junit.Test
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+
+class GroupAggregationsTest extends TableTestBase {
+
+  @Test(expected = classOf[ValidationException])
+  def testGroupingOnNonExistentField(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val ds = table
+             // must fail. '_foo is not a valid field
+             .groupBy('_foo)
+             .select('a.avg)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testGroupingInvalidSelection(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val ds = table
+             .groupBy('a, 'b)
+             // must fail. 'c is not a grouping key or aggregation
+             .select('c)
+  }
+
+  @Test
+  def testGroupbyWithoutWindow() = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val resultTable = table
+                      .groupBy('b)
+                      .select('a.count)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "b")
+          ),
+          term("groupBy", "b"),
+          term("select", "b", "COUNT(a) AS TMP_0")
+        ),
+        term("select", "TMP_0")
+      )
+    util.verifyTable(resultTable, expected)
+  }
+
+
+  @Test
+  def testGroupAggregateWithConstant1(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val resultTable = table
+            .select('a, 4 as 'four, 'b)
+            .groupBy('four, 'a)
+            .select('four, 'b.sum)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "4 AS four", "b", "a")
+          ),
+          term("groupBy", "four", "a"),
+          term("select", "four", "a", "SUM(b) AS TMP_0")
+        ),
+        term("select", "4 AS four", "TMP_0")
+      )
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testGroupAggregateWithConstant2(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val resultTable = table
+            .select('b, 4 as 'four, 'a)
+            .groupBy('b, 'four)
+            .select('four, 'a.sum)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "4 AS four", "a", "b")
+          ),
+          term("groupBy", "four", "b"),
+          term("select", "four", "b", "SUM(a) AS TMP_0")
+        ),
+        term("select", "4 AS four", "TMP_0")
+      )
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testGroupAggregateWithExpressionInSelect(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val resultTable = table
+            .select('a as 'a, 'b % 3 as 'd, 'c as 'c)
+            .groupBy('d)
+            .select('c.min, 'a.avg)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "MOD(b, 3) AS d", "c")
+          ),
+          term("groupBy", "d"),
+          term("select", "d", "MIN(c) AS TMP_0", "AVG(a) AS TMP_1")
+        ),
+        term("select", "TMP_0", "TMP_1")
+      )
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testGroupAggregateWithFilter(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val resultTable = table
+            .groupBy('b)
+            .select('b, 'a.sum)
+            .where('b === 2)
+
+    val expected =
+      unaryNode(
+        "DataStreamGroupAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "b", "a"),
+          term("where", "=(b, 2)")
+        ),
+        term("groupBy", "b"),
+        term("select", "b", "SUM(a) AS TMP_0")
+      )
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testGroupAggregateWithAverage(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('a, 'b, 'c)
+
+    val resultTable = table
+            .groupBy('b)
+            .select('b, 'a.cast(BasicTypeInfo.DOUBLE_TYPE_INFO).avg)
+
+    val expected =
+      unaryNode(
+        "DataStreamGroupAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "b", "a", "CAST(a) AS a0")
+        ),
+        term("groupBy", "b"),
+        term("select", "b", "AVG(a0) AS TMP_0")
+      )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+}


[04/15] flink git commit: [FLINK-5884] [table] Integrate time indicators for Table API & SQL

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
index 0885929..362d846 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
@@ -19,8 +19,9 @@ package org.apache.flink.table.expressions
 
 import org.apache.calcite.rex.RexNode
 import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
 import org.apache.flink.table.api.{UnresolvedException, ValidationException}
+import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
 
 trait NamedExpression extends Expression {
@@ -116,24 +117,6 @@ case class UnresolvedAlias(child: Expression) extends UnaryExpression with Named
   override private[flink] lazy val valid = false
 }
 
-case class RowtimeAttribute() extends Attribute {
-  override private[flink] def withName(newName: String): Attribute = {
-    if (newName == "rowtime") {
-      this
-    } else {
-      throw new ValidationException("Cannot rename streaming rowtime attribute.")
-    }
-  }
-
-  override private[flink] def name: String = "rowtime"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    throw new UnsupportedOperationException("A rowtime attribute can not be used solely.")
-  }
-
-  override private[flink] def resultType: TypeInformation[_] = BasicTypeInfo.LONG_TYPE_INFO
-}
-
 case class WindowReference(name: String) extends Attribute {
 
   override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
@@ -150,3 +133,30 @@ case class WindowReference(name: String) extends Attribute {
     }
   }
 }
+
+abstract class TimeAttribute(val expression: Expression)
+  extends UnaryExpression
+  with NamedExpression {
+
+  override private[flink] def child: Expression = expression
+
+  override private[flink] def name: String = expression match {
+    case UnresolvedFieldReference(name) => name
+    case _ => throw new ValidationException("Unresolved field reference expected.")
+  }
+
+  override private[flink] def toAttribute: Attribute =
+    throw new UnsupportedOperationException("Time attribute can not be used solely.")
+}
+
+case class RowtimeAttribute(expr: Expression) extends TimeAttribute(expr) {
+
+  override private[flink] def resultType: TypeInformation[_] =
+    TimeIndicatorTypeInfo.ROWTIME_INDICATOR
+}
+
+case class ProctimeAttribute(expr: Expression) extends TimeAttribute(expr) {
+
+  override private[flink] def resultType: TypeInformation[_] =
+    TimeIndicatorTypeInfo.PROCTIME_INDICATOR
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala
new file mode 100644
index 0000000..d875026
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala
@@ -0,0 +1,41 @@
+/*
+ * 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.table.functions
+
+import org.apache.calcite.sql._
+import org.apache.calcite.sql.`type`._
+import org.apache.calcite.sql.validate.SqlMonotonicity
+
+/**
+  * Function that materializes a time attribute to the metadata timestamp. After materialization
+  * the result can be used in regular arithmetical calculations.
+  */
+object TimeMaterializationSqlFunction
+  extends SqlFunction(
+    "TIME_MATERIALIZATION",
+    SqlKind.OTHER_FUNCTION,
+    ReturnTypes.explicit(SqlTypeName.TIMESTAMP),
+    InferTypes.RETURN_TYPE,
+    OperandTypes.family(SqlTypeFamily.TIMESTAMP),
+    SqlFunctionCategory.SYSTEM) {
+
+  override def getSyntax: SqlSyntax = SqlSyntax.FUNCTION
+
+  override def getMonotonicity(call: SqlOperatorBinding): SqlMonotonicity =
+    SqlMonotonicity.INCREASING
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeModeIndicatorFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeModeIndicatorFunctions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeModeIndicatorFunctions.scala
deleted file mode 100644
index 3ddcbdc..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeModeIndicatorFunctions.scala
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * 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.table.functions
-
-import java.nio.charset.Charset
-import java.util
-
-import org.apache.calcite.rel.`type`._
-import org.apache.calcite.sql._
-import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes, SqlTypeFamily, SqlTypeName}
-import org.apache.calcite.sql.validate.SqlMonotonicity
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
-import org.apache.flink.table.api.TableException
-import org.apache.flink.table.expressions.LeafExpression
-
-object EventTimeExtractor extends SqlFunction("ROWTIME", SqlKind.OTHER_FUNCTION,
-  ReturnTypes.explicit(TimeModeTypes.ROWTIME), null, OperandTypes.NILADIC,
-  SqlFunctionCategory.SYSTEM) {
-  override def getSyntax: SqlSyntax = SqlSyntax.FUNCTION
-
-  override def getMonotonicity(call: SqlOperatorBinding): SqlMonotonicity =
-    SqlMonotonicity.INCREASING
-}
-
-object ProcTimeExtractor extends SqlFunction("PROCTIME", SqlKind.OTHER_FUNCTION,
-  ReturnTypes.explicit(TimeModeTypes.PROCTIME), null, OperandTypes.NILADIC,
-  SqlFunctionCategory.SYSTEM) {
-  override def getSyntax: SqlSyntax = SqlSyntax.FUNCTION
-
-  override def getMonotonicity(call: SqlOperatorBinding): SqlMonotonicity =
-    SqlMonotonicity.INCREASING
-}
-
-abstract class TimeIndicator extends LeafExpression {
-  /**
-    * Returns the [[org.apache.flink.api.common.typeinfo.TypeInformation]]
-    * for evaluating this expression.
-    * It is sometimes not available until the expression is valid.
-    */
-  override private[flink] def resultType = SqlTimeTypeInfo.TIMESTAMP
-
-  /**
-    * Convert Expression to its counterpart in Calcite, i.e. RexNode
-    */
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder) =
-    throw new TableException("indicator functions (e.g. proctime() and rowtime()" +
-      " are not executable. Please check your expressions.")
-}
-
-case class RowTime() extends TimeIndicator
-case class ProcTime() extends TimeIndicator
-
-object TimeModeTypes {
-
-  // indicator data type for row time (event time)
-  val ROWTIME = new RowTimeType
-  // indicator data type for processing time
-  val PROCTIME = new ProcTimeType
-
-}
-
-class RowTimeType extends TimeModeType {
-
-  override def toString(): String = "ROWTIME"
-  override def getFullTypeString: String = "ROWTIME_INDICATOR"
-}
-
-class ProcTimeType extends TimeModeType {
-
-  override def toString(): String = "PROCTIME"
-  override def getFullTypeString: String = "PROCTIME_INDICATOR"
-}
-
-abstract class TimeModeType extends RelDataType {
-
-  override def getComparability: RelDataTypeComparability = RelDataTypeComparability.NONE
-
-  override def isStruct: Boolean = false
-
-  override def getFieldList: util.List[RelDataTypeField] = null
-
-  override def getFieldNames: util.List[String] = null
-
-  override def getFieldCount: Int = 0
-
-  override def getStructKind: StructKind = StructKind.NONE
-
-  override def getField(
-     fieldName: String,
-     caseSensitive: Boolean,
-     elideRecord: Boolean): RelDataTypeField = null
-
-  override def isNullable: Boolean = false
-
-  override def getComponentType: RelDataType = null
-
-  override def getKeyType: RelDataType = null
-
-  override def getValueType: RelDataType = null
-
-  override def getCharset: Charset = null
-
-  override def getCollation: SqlCollation = null
-
-  override def getIntervalQualifier: SqlIntervalQualifier = null
-
-  override def getPrecision: Int = -1
-
-  override def getScale: Int = -1
-
-  override def getSqlTypeName: SqlTypeName = SqlTypeName.TIMESTAMP
-
-  override def getSqlIdentifier: SqlIdentifier = null
-
-  override def getFamily: RelDataTypeFamily = SqlTypeFamily.NUMERIC
-
-  override def getPrecedenceList: RelDataTypePrecedenceList = ???
-
-  override def isDynamicStruct: Boolean = false
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
index d26cdcf..98a7e63 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
@@ -19,9 +19,8 @@
 package org.apache.flink.table.plan
 
 import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.table.api.{OverWindow, StreamTableEnvironment, TableEnvironment}
+import org.apache.flink.table.api.{OverWindow, TableEnvironment}
 import org.apache.flink.table.expressions._
-import org.apache.flink.table.functions.{ProcTime, RowTime}
 import org.apache.flink.table.plan.logical.{LogicalNode, Project}
 
 import scala.collection.mutable
@@ -231,28 +230,12 @@ object ProjectionTranslator {
 
       val overWindow = overWindows.find(_.alias.equals(unresolvedCall.alias))
       if (overWindow.isDefined) {
-        if (tEnv.isInstanceOf[StreamTableEnvironment]) {
-          val timeIndicator = overWindow.get.orderBy match {
-            case u: UnresolvedFieldReference if u.name.toLowerCase == "rowtime" =>
-              RowTime()
-            case u: UnresolvedFieldReference if u.name.toLowerCase == "proctime" =>
-              ProcTime()
-            case e: Expression => e
-          }
-          OverCall(
-            unresolvedCall.agg,
-            overWindow.get.partitionBy,
-            timeIndicator,
-            overWindow.get.preceding,
-            overWindow.get.following)
-        } else {
-          OverCall(
-            unresolvedCall.agg,
-            overWindow.get.partitionBy,
-            overWindow.get.orderBy,
-            overWindow.get.preceding,
-            overWindow.get.following)
-        }
+        OverCall(
+          unresolvedCall.agg,
+          overWindow.get.partitionBy,
+          overWindow.get.orderBy,
+          overWindow.get.preceding,
+          overWindow.get.following)
       } else {
         unresolvedCall
       }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
index 1884e54..92dc501 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
@@ -22,14 +22,24 @@ import org.apache.flink.table.api.TableEnvironment
 import org.apache.flink.table.expressions.{Expression, WindowReference}
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
 
-abstract class LogicalWindow(val alias: Expression) extends Resolvable[LogicalWindow] {
+/**
+  * Logical super class for all types of windows (group-windows and row-windows).
+  *
+  * @param aliasAttribute window alias
+  * @param timeAttribute time field indicating event-time or processing-time
+  */
+abstract class LogicalWindow(
+    val aliasAttribute: Expression,
+    val timeAttribute: Expression)
+  extends Resolvable[LogicalWindow] {
 
   def resolveExpressions(resolver: (Expression) => Expression): LogicalWindow = this
 
-  def validate(tableEnv: TableEnvironment): ValidationResult = alias match {
+  def validate(tableEnv: TableEnvironment): ValidationResult = aliasAttribute match {
     case WindowReference(_) => ValidationSuccess
     case _ => ValidationFailure("Window reference for window expected.")
   }
 
   override def toString: String = getClass.getSimpleName
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
index 576756d..3e5de28 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
@@ -18,259 +18,165 @@
 
 package org.apache.flink.table.plan.logical
 
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
 import org.apache.flink.table.api.{BatchTableEnvironment, StreamTableEnvironment, TableEnvironment}
+import org.apache.flink.table.expressions.ExpressionUtils.{isRowCountLiteral, isRowtimeAttribute, isTimeAttribute, isTimeIntervalLiteral}
 import org.apache.flink.table.expressions._
-import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo, TypeCoercion}
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimePoint
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
 
-abstract class EventTimeGroupWindow(
-    alias: Expression,
-    time: Expression)
-  extends LogicalWindow(alias) {
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult = {
-    val valid = super.validate(tableEnv)
-    if (valid.isFailure) {
-        return valid
-    }
-
-    tableEnv match {
-      case _: StreamTableEnvironment =>
-        time match {
-          case RowtimeAttribute() =>
-            ValidationSuccess
-          case _ =>
-            ValidationFailure("Event-time window expects a 'rowtime' time field.")
-      }
-      case _: BatchTableEnvironment =>
-        if (!TypeCoercion.canCast(time.resultType, BasicTypeInfo.LONG_TYPE_INFO)) {
-          ValidationFailure(s"Event-time window expects a time field that can be safely cast " +
-            s"to Long, but is ${time.resultType}")
-        } else {
-          ValidationSuccess
-        }
-    }
-
-  }
-}
-
-abstract class ProcessingTimeGroupWindow(alias: Expression) extends LogicalWindow(alias) {
-  override def validate(tableEnv: TableEnvironment): ValidationResult = {
-    val valid = super.validate(tableEnv)
-    if (valid.isFailure) {
-      return valid
-    }
-
-    tableEnv match {
-      case b: BatchTableEnvironment => ValidationFailure(
-        "Window on batch must declare a time attribute over which the query is evaluated.")
-      case _ =>
-        ValidationSuccess
-    }
-  }
-}
-
 // ------------------------------------------------------------------------------------------------
 // Tumbling group windows
 // ------------------------------------------------------------------------------------------------
 
-object TumblingGroupWindow {
-  def validate(tableEnv: TableEnvironment, size: Expression): ValidationResult = size match {
-    case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-      ValidationSuccess
-    case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-      ValidationSuccess
-    case _ =>
-      ValidationFailure("Tumbling window expects size literal of type Interval of Milliseconds " +
-        "or Interval of Rows.")
-  }
-}
-
-case class ProcessingTimeTumblingGroupWindow(
-    override val alias: Expression,
-    size: Expression)
-  extends ProcessingTimeGroupWindow(alias) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    ProcessingTimeTumblingGroupWindow(
-      resolve(alias),
-      resolve(size))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(TumblingGroupWindow.validate(tableEnv, size))
-
-  override def toString: String = s"ProcessingTimeTumblingGroupWindow($alias, $size)"
-}
-
-case class EventTimeTumblingGroupWindow(
-    override val alias: Expression,
+case class TumblingGroupWindow(
+    alias: Expression,
     timeField: Expression,
     size: Expression)
-  extends EventTimeGroupWindow(
+  extends LogicalWindow(
     alias,
     timeField) {
 
   override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    EventTimeTumblingGroupWindow(
+    TumblingGroupWindow(
       resolve(alias),
       resolve(timeField),
       resolve(size))
 
   override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv)
-      .orElse(TumblingGroupWindow.validate(tableEnv, size))
-      .orElse(size match {
-        case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS)
-          if tableEnv.isInstanceOf[StreamTableEnvironment] =>
+    super.validate(tableEnv).orElse(
+      tableEnv match {
+
+        // check size
+        case _ if !isTimeIntervalLiteral(size) && !isRowCountLiteral(size) =>
+          ValidationFailure(
+            "Tumbling window expects size literal of type Interval of Milliseconds " +
+              "or Interval of Rows.")
+
+        // check time attribute
+        case _: StreamTableEnvironment if !isTimeAttribute(timeField) =>
+          ValidationFailure(
+            "Tumbling window expects a time attribute for grouping in a stream environment.")
+        case _: BatchTableEnvironment if isTimePoint(size.resultType) =>
+          ValidationFailure(
+            "Tumbling window expects a time attribute for grouping in a stream environment.")
+
+        // check row intervals on event-time
+        case _: StreamTableEnvironment
+            if isRowCountLiteral(size) && isRowtimeAttribute(timeField) =>
           ValidationFailure(
             "Event-time grouping windows on row intervals in a stream environment " +
               "are currently not supported.")
+
         case _ =>
           ValidationSuccess
-      })
+      }
+    )
 
-  override def toString: String = s"EventTimeTumblingGroupWindow($alias, $timeField, $size)"
+  override def toString: String = s"TumblingGroupWindow($alias, $timeField, $size)"
 }
 
 // ------------------------------------------------------------------------------------------------
 // Sliding group windows
 // ------------------------------------------------------------------------------------------------
 
-object SlidingGroupWindow {
-  def validate(
-      tableEnv: TableEnvironment,
-      size: Expression,
-      slide: Expression)
-    : ValidationResult = {
-
-    val checkedSize = size match {
-      case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        ValidationSuccess
-      case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-        ValidationSuccess
-      case _ =>
-        ValidationFailure("Sliding window expects size literal of type Interval of " +
-          "Milliseconds or Interval of Rows.")
-    }
-
-    val checkedSlide = slide match {
-      case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        ValidationSuccess
-      case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-        ValidationSuccess
-      case _ =>
-        ValidationFailure("Sliding window expects slide literal of type Interval of " +
-          "Milliseconds or Interval of Rows.")
-    }
-
-    checkedSize
-      .orElse(checkedSlide)
-      .orElse {
-        if (size.resultType != slide.resultType) {
-          ValidationFailure("Sliding window expects same type of size and slide.")
-        } else {
-          ValidationSuccess
-        }
-      }
-  }
-}
-
-case class ProcessingTimeSlidingGroupWindow(
-    override val alias: Expression,
+case class SlidingGroupWindow(
+    alias: Expression,
+    timeField: Expression,
     size: Expression,
     slide: Expression)
-  extends ProcessingTimeGroupWindow(alias) {
+  extends LogicalWindow(
+    alias,
+    timeField) {
 
   override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    ProcessingTimeSlidingGroupWindow(
+    SlidingGroupWindow(
       resolve(alias),
+      resolve(timeField),
       resolve(size),
       resolve(slide))
 
   override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(SlidingGroupWindow.validate(tableEnv, size, slide))
+    super.validate(tableEnv).orElse(
+      tableEnv match {
 
-  override def toString: String = s"ProcessingTimeSlidingGroupWindow($alias, $size, $slide)"
-}
+        // check size
+        case _ if !isTimeIntervalLiteral(size) && !isRowCountLiteral(size) =>
+          ValidationFailure(
+            "Sliding window expects size literal of type Interval of Milliseconds " +
+              "or Interval of Rows.")
 
-case class EventTimeSlidingGroupWindow(
-    override val alias: Expression,
-    timeField: Expression,
-    size: Expression,
-    slide: Expression)
-  extends EventTimeGroupWindow(alias, timeField) {
+        // check slide
+        case _ if !isTimeIntervalLiteral(slide) && !isRowCountLiteral(slide) =>
+          ValidationFailure(
+            "Sliding window expects slide literal of type Interval of Milliseconds " +
+              "or Interval of Rows.")
 
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    EventTimeSlidingGroupWindow(
-      resolve(alias),
-      resolve(timeField),
-      resolve(size),
-      resolve(slide))
+        // check same type of intervals
+        case _ if isTimeIntervalLiteral(size) != isTimeIntervalLiteral(slide) =>
+          ValidationFailure("Sliding window expects same type of size and slide.")
 
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv)
-      .orElse(SlidingGroupWindow.validate(tableEnv, size, slide))
-      .orElse(size match {
-        case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS)
-          if tableEnv.isInstanceOf[StreamTableEnvironment] =>
+        // check time attribute
+        case _: StreamTableEnvironment if !isTimeAttribute(timeField) =>
+          ValidationFailure(
+            "Sliding window expects a time attribute for grouping in a stream environment.")
+        case _: BatchTableEnvironment if isTimePoint(size.resultType) =>
+          ValidationFailure(
+            "Sliding window expects a time attribute for grouping in a stream environment.")
+
+        // check row intervals on event-time
+        case _: StreamTableEnvironment
+            if isRowCountLiteral(size) && isRowtimeAttribute(timeField) =>
           ValidationFailure(
             "Event-time grouping windows on row intervals in a stream environment " +
               "are currently not supported.")
+
         case _ =>
           ValidationSuccess
-      })
+      }
+    )
 
-  override def toString: String = s"EventTimeSlidingGroupWindow($alias, $timeField, $size, $slide)"
+  override def toString: String = s"SlidingGroupWindow($alias, $timeField, $size, $slide)"
 }
 
 // ------------------------------------------------------------------------------------------------
 // Session group windows
 // ------------------------------------------------------------------------------------------------
 
-object SessionGroupWindow {
-
-  def validate(tableEnv: TableEnvironment, gap: Expression): ValidationResult = gap match {
-    case Literal(timeInterval: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-      ValidationSuccess
-    case _ =>
-      ValidationFailure(
-        "Session window expects gap literal of type Interval of Milliseconds.")
-  }
-}
-
-case class ProcessingTimeSessionGroupWindow(
-    override val alias: Expression,
-    gap: Expression)
-  extends ProcessingTimeGroupWindow(alias) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    ProcessingTimeSessionGroupWindow(
-      resolve(alias),
-      resolve(gap))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(SessionGroupWindow.validate(tableEnv, gap))
-
-  override def toString: String = s"ProcessingTimeSessionGroupWindow($alias, $gap)"
-}
-
-case class EventTimeSessionGroupWindow(
-    override val alias: Expression,
+case class SessionGroupWindow(
+    alias: Expression,
     timeField: Expression,
     gap: Expression)
-  extends EventTimeGroupWindow(
+  extends LogicalWindow(
     alias,
     timeField) {
 
   override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    EventTimeSessionGroupWindow(
+    SessionGroupWindow(
       resolve(alias),
       resolve(timeField),
       resolve(gap))
 
   override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(SessionGroupWindow.validate(tableEnv, gap))
+    super.validate(tableEnv).orElse(
+      tableEnv match {
+
+        // check size
+        case _ if !isTimeIntervalLiteral(gap) =>
+          ValidationFailure(
+            "Session window expects size literal of type Interval of Milliseconds.")
+
+        // check time attribute
+        case _: StreamTableEnvironment if !isTimeAttribute(timeField) =>
+          ValidationFailure(
+            "Session window expects a time attribute for grouping in a stream environment.")
+        case _: BatchTableEnvironment if isTimePoint(gap.resultType) =>
+          ValidationFailure(
+            "Session window expects a time attribute for grouping in a stream environment.")
+
+        case _ =>
+          ValidationSuccess
+      }
+    )
 
-  override def toString: String = s"EventTimeSessionGroupWindow($alias, $timeField, $gap)"
+  override def toString: String = s"SessionGroupWindow($alias, $timeField, $gap)"
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/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 5f2394c..3839145 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
@@ -70,8 +70,6 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extend
     def checkName(name: String): Unit = {
       if (names.contains(name)) {
         failValidation(s"Duplicate field name $name.")
-      } else if (tableEnv.isInstanceOf[StreamTableEnvironment] && name == "rowtime") {
-        failValidation("'rowtime' cannot be used as field name in a streaming environment.")
       } else {
         names.add(name)
       }
@@ -112,10 +110,6 @@ case class AliasNode(aliasList: Seq[Expression], child: LogicalNode) extends Una
       failValidation("Alias only accept name expressions as arguments")
     } else if (!aliasList.forall(_.asInstanceOf[UnresolvedFieldReference].name != "*")) {
       failValidation("Alias can not accept '*' as name")
-    } else if (tableEnv.isInstanceOf[StreamTableEnvironment] && !aliasList.forall {
-          case UnresolvedFieldReference(name) => name != "rowtime"
-        }) {
-      failValidation("'rowtime' cannot be used as field name in a streaming environment.")
     } else {
       val names = aliasList.map(_.asInstanceOf[UnresolvedFieldReference].name)
       val input = child.output
@@ -561,26 +555,20 @@ case class WindowAggregate(
   override def resolveReference(
       tableEnv: TableEnvironment,
       name: String)
-    : Option[NamedExpression] = tableEnv match {
-    // resolve reference to rowtime attribute in a streaming environment
-    case _: StreamTableEnvironment if name == "rowtime" =>
-      Some(RowtimeAttribute())
-    case _ =>
-      window.alias match {
-        // resolve reference to this window's alias
-        case UnresolvedFieldReference(alias) if name == alias =>
-          // check if reference can already be resolved by input fields
-          val found = super.resolveReference(tableEnv, name)
-          if (found.isDefined) {
-            failValidation(s"Reference $name is ambiguous.")
-          } else {
-            Some(WindowReference(name))
-          }
-        case _ =>
-          // resolve references as usual
-          super.resolveReference(tableEnv, name)
-      }
-  }
+    : Option[NamedExpression] = window.aliasAttribute match {
+      // resolve reference to this window's name
+      case UnresolvedFieldReference(alias) if name == alias =>
+        // check if reference can already be resolved by input fields
+        val found = super.resolveReference(tableEnv, name)
+        if (found.isDefined) {
+          failValidation(s"Reference $name is ambiguous.")
+        } else {
+          Some(WindowReference(name))
+        }
+      case _ =>
+        // resolve references as usual
+        super.resolveReference(tableEnv, name)
+    }
 
   override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
     val flinkRelBuilder = relBuilder.asInstanceOf[FlinkRelBuilder]

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
index 96a7470..5c35129 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
@@ -19,13 +19,12 @@
 package org.apache.flink.table.plan.nodes
 
 import org.apache.calcite.plan.{RelOptCost, RelOptPlanner}
-import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rex._
 import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.{CodeGenerator, GeneratedFunction}
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.FlatMapRunner
 import org.apache.flink.types.Row
 
@@ -35,21 +34,30 @@ import scala.collection.JavaConverters._
 trait CommonCalc {
 
   private[flink] def functionBody(
-      generator: CodeGenerator,
-      inputType: TypeInformation[Row],
-      rowType: RelDataType,
-      calcProgram: RexProgram,
-      config: TableConfig)
+     generator: CodeGenerator,
+     inputSchema: RowSchema,
+     returnSchema: RowSchema,
+     calcProgram: RexProgram,
+     config: TableConfig)
     : String = {
 
-    val returnType = FlinkTypeFactory.toInternalRowTypeInfo(rowType)
+    val expandedExpressions = calcProgram
+      .getProjectList
+      .map(expr => calcProgram.expandLocalRef(expr))
+      // time indicator fields must not be part of the code generation
+      .filter(expr => !FlinkTypeFactory.isTimeIndicatorType(expr.getType))
+      // update indices
+      .map(expr => inputSchema.mapRexNode(expr))
+
+    val condition = if (calcProgram.getCondition != null) {
+      inputSchema.mapRexNode(calcProgram.expandLocalRef(calcProgram.getCondition))
+    } else {
+      null
+    }
 
-    val condition = calcProgram.getCondition
-    val expandedExpressions = calcProgram.getProjectList.map(
-      expr => calcProgram.expandLocalRef(expr))
     val projection = generator.generateResultExpression(
-      returnType,
-      rowType.getFieldNames,
+      returnSchema.physicalTypeInfo,
+      returnSchema.physicalFieldNames,
       expandedExpressions)
 
     // only projection
@@ -60,8 +68,7 @@ trait CommonCalc {
         |""".stripMargin
     }
     else {
-      val filterCondition = generator.generateExpression(
-        calcProgram.expandLocalRef(calcProgram.getCondition))
+      val filterCondition = generator.generateExpression(condition)
       // only filter
       if (projection == null) {
         s"""

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
index 6c4066b..02305ee 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
@@ -23,11 +23,11 @@ import org.apache.calcite.sql.SemiJoinType
 import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.{TableConfig, TableException}
-import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenUtils.primitiveDefaultValue
 import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NO_CODE}
 import org.apache.flink.table.codegen.{CodeGenerator, GeneratedCollector, GeneratedExpression, GeneratedFunction}
 import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.{CorrelateFlatMapRunner, TableFunctionCollector}
 import org.apache.flink.types.Row
 
@@ -44,9 +44,9 @@ trait CommonCorrelate {
     */
   private[flink] def correlateMapFunction(
       config: TableConfig,
-      inputTypeInfo: TypeInformation[Row],
+      inputSchema: RowSchema,
       udtfTypeInfo: TypeInformation[Any],
-      rowType: RelDataType,
+      returnSchema: RowSchema,
       joinType: SemiJoinType,
       rexCall: RexCall,
       condition: Option[RexNode],
@@ -54,26 +54,24 @@ trait CommonCorrelate {
       ruleDescription: String)
     : CorrelateFlatMapRunner[Row, Row] = {
 
-    val returnType = FlinkTypeFactory.toInternalRowTypeInfo(rowType)
-
     val flatMap = generateFunction(
       config,
-      inputTypeInfo,
+      inputSchema.physicalTypeInfo,
       udtfTypeInfo,
-      returnType,
-      rowType,
+      returnSchema.physicalTypeInfo,
+      returnSchema.logicalFieldNames,
       joinType,
-      rexCall,
+      inputSchema.mapRexNode(rexCall).asInstanceOf[RexCall],
       pojoFieldMapping,
       ruleDescription)
 
     val collector = generateCollector(
       config,
-      inputTypeInfo,
+      inputSchema.physicalTypeInfo,
       udtfTypeInfo,
-      returnType,
-      rowType,
-      condition,
+      returnSchema.physicalTypeInfo,
+      returnSchema.logicalFieldNames,
+      condition.map(inputSchema.mapRexNode),
       pojoFieldMapping)
 
     new CorrelateFlatMapRunner[Row, Row](
@@ -93,7 +91,7 @@ trait CommonCorrelate {
       inputTypeInfo: TypeInformation[Row],
       udtfTypeInfo: TypeInformation[Any],
       returnType: TypeInformation[Row],
-      rowType: RelDataType,
+      resultFieldNames: Seq[String],
       joinType: SemiJoinType,
       rexCall: RexCall,
       pojoFieldMapping: Option[Array[Int]],
@@ -134,7 +132,7 @@ trait CommonCorrelate {
           x.resultType)
       }
       val outerResultExpr = functionGenerator.generateResultExpression(
-        input1AccessExprs ++ input2NullExprs, returnType, rowType.getFieldNames.asScala)
+        input1AccessExprs ++ input2NullExprs, returnType, resultFieldNames)
       body +=
         s"""
           |boolean hasOutput = $collectorTerm.isCollected();
@@ -162,7 +160,7 @@ trait CommonCorrelate {
       inputTypeInfo: TypeInformation[Row],
       udtfTypeInfo: TypeInformation[Any],
       returnType: TypeInformation[Row],
-      rowType: RelDataType,
+      resultFieldNames: Seq[String],
       condition: Option[RexNode],
       pojoFieldMapping: Option[Array[Int]])
     : GeneratedCollector = {
@@ -180,7 +178,7 @@ trait CommonCorrelate {
     val crossResultExpr = generator.generateResultExpression(
       input1AccessExprs ++ input2AccessExprs,
       returnType,
-      rowType.getFieldNames.asScala)
+      resultFieldNames)
 
     val collectorCode = if (condition.isEmpty) {
       s"""

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
index 0a0d204..091a1ea 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
@@ -18,11 +18,10 @@
 
 package org.apache.flink.table.plan.nodes
 
-import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.functions.Function
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.TableConfig
-import org.apache.flink.table.codegen.CodeGenerator
-import org.apache.flink.table.runtime.MapRunner
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedFunction}
 import org.apache.flink.types.Row
 
 /**
@@ -42,21 +41,22 @@ trait CommonScan {
     externalTypeInfo != internalTypeInfo
   }
 
-  private[flink] def getConversionMapper(
+  private[flink] def generatedConversionFunction[F <: Function](
       config: TableConfig,
+      functionClass: Class[F],
       inputType: TypeInformation[Any],
       expectedType: TypeInformation[Row],
       conversionOperatorName: String,
       fieldNames: Seq[String],
-      inputPojoFieldMapping: Option[Array[Int]] = None)
-    : MapFunction[Any, Row] = {
+      inputFieldMapping: Option[Array[Int]] = None)
+    : GeneratedFunction[F, Row] = {
 
     val generator = new CodeGenerator(
       config,
       false,
       inputType,
       None,
-      inputPojoFieldMapping)
+      inputFieldMapping)
     val conversion = generator.generateConverterResultExpression(expectedType, fieldNames)
 
     val body =
@@ -65,17 +65,11 @@ trait CommonScan {
          |return ${conversion.resultTerm};
          |""".stripMargin
 
-    val genFunction = generator.generateFunction(
+    generator.generateFunction(
       conversionOperatorName,
-      classOf[MapFunction[Any, Row]],
+      functionClass,
       body,
       expectedType)
-
-    new MapRunner[Any, Row](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
index 6878473..1048549 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
@@ -18,14 +18,13 @@
 
 package org.apache.flink.table.plan.nodes
 
-import org.apache.calcite.rel.{RelFieldCollation, RelNode}
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl}
-import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.{AggregateCall, Window}
 import org.apache.calcite.rel.core.Window.Group
-import org.apache.calcite.rel.core.Window
-import org.apache.calcite.rex.{RexInputRef}
+import org.apache.calcite.rel.{RelFieldCollation, RelNode}
+import org.apache.calcite.rex.RexInputRef
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.aggregate.AggregateUtil._
-import org.apache.flink.table.functions.{ProcTimeType, RowTimeType}
 
 import scala.collection.JavaConverters._
 
@@ -43,7 +42,7 @@ trait OverAggregate {
     val inFields = inputType.getFieldList.asScala
 
     val orderingString = orderFields.asScala.map {
-      x => inFields(x.getFieldIndex).getValue
+      x => inFields(x.getFieldIndex).getName
     }.mkString(", ")
 
     orderingString
@@ -66,24 +65,8 @@ trait OverAggregate {
     rowType: RelDataType,
     namedAggregates: Seq[CalcitePair[AggregateCall, String]]): String = {
 
-    val inFields = inputType.getFieldList.asScala.map {
-      x =>
-        x.asInstanceOf[RelDataTypeFieldImpl].getType
-        match {
-          case proceTime: ProcTimeType => "PROCTIME"
-          case rowTime: RowTimeType => "ROWTIME"
-          case _ => x.asInstanceOf[RelDataTypeFieldImpl].getName
-        }
-    }
-    val outFields = rowType.getFieldList.asScala.map {
-      x =>
-        x.asInstanceOf[RelDataTypeFieldImpl].getType
-        match {
-          case proceTime: ProcTimeType => "PROCTIME"
-          case rowTime: RowTimeType => "ROWTIME"
-          case _ => x.asInstanceOf[RelDataTypeFieldImpl].getName
-        }
-    }
+    val inFields = inputType.getFieldNames.asScala
+    val outFields = rowType.getFieldNames.asScala
 
     val aggStrings = namedAggregates.map(_.getKey).map(
       a => s"${a.getAggregation}(${
@@ -109,7 +92,7 @@ trait OverAggregate {
     input: RelNode): Long = {
 
     val ref: RexInputRef = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef]
-    val lowerBoundIndex = input.getRowType.getFieldCount - ref.getIndex;
+    val lowerBoundIndex = input.getRowType.getFieldCount - ref.getIndex
     val lowerBound = logicWindow.constants.get(lowerBoundIndex).getValue2
     lowerBound match {
       case x: java.math.BigDecimal => x.asInstanceOf[java.math.BigDecimal].longValue()

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
index d924450..c18c3d1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala
@@ -37,9 +37,11 @@ abstract class PhysicalTableSourceScan(
 
   override def deriveRowType(): RelDataType = {
     val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    flinkTypeFactory.buildRowDataType(
+    flinkTypeFactory.buildLogicalRowType(
       TableEnvironment.getFieldNames(tableSource),
-      TableEnvironment.getFieldTypes(tableSource.getReturnType))
+      TableEnvironment.getFieldTypes(tableSource.getReturnType),
+      None,
+      None)
   }
 
   override def explainTerms(pw: RelWriter): RelWriter = {

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
index b39b8ed..cc5d9fb 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
@@ -18,11 +18,13 @@
 
 package org.apache.flink.table.plan.nodes.dataset
 
+import org.apache.flink.api.common.functions.MapFunction
 import org.apache.flink.api.java.DataSet
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.CommonScan
 import org.apache.flink.table.plan.schema.FlinkTable
+import org.apache.flink.table.runtime.MapRunner
 import org.apache.flink.types.Row
 
 import scala.collection.JavaConversions._
@@ -43,17 +45,23 @@ trait BatchScan extends CommonScan with DataSetRel {
     // conversion
     if (needsConversion(inputType, internalType)) {
 
-      val mapFunc = getConversionMapper(
+      val function = generatedConversionFunction(
         config,
+        classOf[MapFunction[Any, Row]],
         inputType,
         internalType,
         "DataSetSourceConversion",
         getRowType.getFieldNames,
         Some(flinkTable.fieldIndexes))
 
+      val runner = new MapRunner[Any, Row](
+        function.name,
+        function.code,
+        function.returnType)
+
       val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
 
-      input.map(mapFunc).name(opName)
+      input.map(runner).name(opName)
     }
     // no conversion necessary, forward
     else {

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
index bf4291a..fb291e4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
@@ -22,7 +22,8 @@ import org.apache.calcite.plan._
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.flink.api.java.DataSet
-import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.api.{BatchTableEnvironment, TableEnvironment}
+import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.PhysicalTableSourceScan
 import org.apache.flink.table.plan.schema.TableSourceTable
 import org.apache.flink.table.sources.{BatchTableSource, TableSource}
@@ -37,7 +38,16 @@ class BatchTableSourceScan(
   extends PhysicalTableSourceScan(cluster, traitSet, table, tableSource)
   with BatchScan {
 
-  override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+  override def deriveRowType() = {
+    val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    flinkTypeFactory.buildLogicalRowType(
+      TableEnvironment.getFieldNames(tableSource),
+      TableEnvironment.getFieldTypes(tableSource.getReturnType),
+      None,
+      None)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
     val rowCnt = metadata.getRowCount(this)
     planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType))
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
index b92775c..c22dc54 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
@@ -91,6 +91,9 @@ class DataSetAggregate(
   override def translateToPlan(tableEnv: BatchTableEnvironment): DataSet[Row] = {
 
     val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+    val input = inputNode.asInstanceOf[DataSetRel]
+
+    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
 
     val generator = new CodeGenerator(
       tableEnv.getConfig,
@@ -104,15 +107,14 @@ class DataSetAggregate(
       ) = AggregateUtil.createDataSetAggregateFunctions(
         generator,
         namedAggregates,
-        inputType,
+        input.getRowType,
+        inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
         rowRelDataType,
         grouping,
         inGroupingSet)
 
     val aggString = aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil)
 
-    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
-
     if (grouping.length > 0) {
       // grouped aggregation
       val aggOpName = s"groupBy: (${groupingToString(inputType, grouping)}), " +

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
index e05b5a8..9e18082 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
@@ -26,10 +26,13 @@ import org.apache.calcite.rel.{RelNode, RelWriter}
 import org.apache.calcite.rex._
 import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.api.java.DataSet
+import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.BatchTableEnvironment
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.nodes.CommonCalc
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.FlatMapRunner
 import org.apache.flink.types.Row
 
 /**
@@ -83,14 +86,14 @@ class DataSetCalc(
 
     val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
 
-    val returnType = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
-
     val generator = new CodeGenerator(config, false, inputDS.getType)
 
+    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
     val body = functionBody(
       generator,
-      inputDS.getType,
-      getRowType,
+      new RowSchema(getInput.getRowType),
+      new RowSchema(getRowType),
       calcProgram,
       config)
 
@@ -98,9 +101,13 @@ class DataSetCalc(
       ruleDescription,
       classOf[FlatMapFunction[Row, Row]],
       body,
-      returnType)
+      rowTypeInfo)
+
+    val runner = new FlatMapRunner[Row, Row](
+      genFunction.name,
+      genFunction.code,
+      genFunction.returnType)
 
-    val mapFunc = calcMapFunction(genFunction)
-    inputDS.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
+    inputDS.flatMap(runner).name(calcOpName(calcProgram, getExpressionString))
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
index 2a62e21..6c79b45 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
@@ -25,10 +25,13 @@ import org.apache.calcite.rex.{RexCall, RexNode}
 import org.apache.calcite.sql.SemiJoinType
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.DataSet
+import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.functions.utils.TableSqlFunction
 import org.apache.flink.table.plan.nodes.CommonCorrelate
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.types.Row
 
 /**
@@ -98,11 +101,13 @@ class DataSetCorrelate(
     val pojoFieldMapping = sqlFunction.getPojoFieldMapping
     val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
 
+    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
     val mapFunc = correlateMapFunction(
       config,
-      inputDS.getType,
+      new RowSchema(getInput.getRowType),
       udtfTypeInfo,
-      getRowType,
+      new RowSchema(getRowType),
       joinType,
       rexCall,
       condition,

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala
index 96c427e..3cb872a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala
@@ -24,15 +24,16 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
 import org.apache.flink.api.common.operators.Order
 import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, RowTypeInfo}
 import org.apache.flink.table.api.BatchTableEnvironment
 import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.expressions.ExpressionUtils._
 import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.plan.nodes.CommonAggregate
 import org.apache.flink.table.runtime.aggregate.AggregateUtil.{CalcitePair, _}
-import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.TypeCheckUtils.{isLong, isTimePoint}
 import org.apache.flink.types.Row
 
 /**
@@ -106,8 +107,6 @@ class DataSetWindowAggregate(
 
   override def translateToPlan(tableEnv: BatchTableEnvironment): DataSet[Row] = {
 
-    val config = tableEnv.getConfig
-
     val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
 
     val generator = new CodeGenerator(
@@ -119,30 +118,31 @@ class DataSetWindowAggregate(
     val caseSensitive = tableEnv.getFrameworkConfig.getParserConfig.caseSensitive()
 
     window match {
-      case EventTimeTumblingGroupWindow(_, _, size) =>
+      case TumblingGroupWindow(_, timeField, size)
+          if isTimePoint(timeField.resultType) || isLong(timeField.resultType) =>
         createEventTimeTumblingWindowDataSet(
           generator,
           inputDS,
-          isTimeInterval(size.resultType),
+          isTimeIntervalLiteral(size),
           caseSensitive)
 
-      case EventTimeSessionGroupWindow(_, _, gap) =>
+      case SessionGroupWindow(_, timeField, gap)
+          if isTimePoint(timeField.resultType) || isLong(timeField.resultType) =>
         createEventTimeSessionWindowDataSet(generator, inputDS, caseSensitive)
 
-      case EventTimeSlidingGroupWindow(_, _, size, slide) =>
+      case SlidingGroupWindow(_, timeField, size, slide)
+          if isTimePoint(timeField.resultType) || isLong(timeField.resultType) =>
         createEventTimeSlidingWindowDataSet(
           generator,
           inputDS,
-          isTimeInterval(size.resultType),
+          isTimeIntervalLiteral(size),
           asLong(size),
           asLong(slide),
           caseSensitive)
 
-      case _: ProcessingTimeGroupWindow =>
+      case _ =>
         throw new UnsupportedOperationException(
-          "Processing-time tumbling windows are not supported in a batch environment, " +
-            "windows in a batch environment must declare a time attribute over which " +
-            "the query is evaluated.")
+          s"Window $window is not supported in a batch environment.")
     }
   }
 
@@ -152,18 +152,22 @@ class DataSetWindowAggregate(
       isTimeWindow: Boolean,
       isParserCaseSensitive: Boolean): DataSet[Row] = {
 
+    val input = inputNode.asInstanceOf[DataSetRel]
+
     val mapFunction = createDataSetWindowPrepareMapFunction(
       generator,
       window,
       namedAggregates,
       grouping,
-      inputType,
+      input.getRowType,
+      inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
       isParserCaseSensitive)
     val groupReduceFunction = createDataSetWindowAggregationGroupReduceFunction(
       generator,
       window,
       namedAggregates,
-      inputType,
+      input.getRowType,
+      inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
       getRowType,
       grouping,
       namedProperties)
@@ -210,6 +214,8 @@ class DataSetWindowAggregate(
       inputDS: DataSet[Row],
       isParserCaseSensitive: Boolean): DataSet[Row] = {
 
+    val input = inputNode.asInstanceOf[DataSetRel]
+
     val groupingKeys = grouping.indices.toArray
     val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
 
@@ -219,7 +225,8 @@ class DataSetWindowAggregate(
       window,
       namedAggregates,
       grouping,
-      inputType,
+      input.getRowType,
+      inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
       isParserCaseSensitive)
 
     val mappedInput = inputDS.map(mapFunction).name(prepareOperatorName)
@@ -245,7 +252,8 @@ class DataSetWindowAggregate(
           generator,
           window,
           namedAggregates,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           grouping)
 
         // create groupReduceFunction for calculating the aggregations
@@ -253,7 +261,8 @@ class DataSetWindowAggregate(
           generator,
           window,
           namedAggregates,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           rowRelDataType,
           grouping,
           namedProperties,
@@ -275,7 +284,8 @@ class DataSetWindowAggregate(
           generator,
           window,
           namedAggregates,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           grouping)
 
         // create groupReduceFunction for calculating the aggregations
@@ -283,7 +293,8 @@ class DataSetWindowAggregate(
           generator,
           window,
           namedAggregates,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           rowRelDataType,
           grouping,
           namedProperties,
@@ -308,7 +319,8 @@ class DataSetWindowAggregate(
           generator,
           window,
           namedAggregates,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           rowRelDataType,
           grouping,
           namedProperties)
@@ -324,7 +336,8 @@ class DataSetWindowAggregate(
           generator,
           window,
           namedAggregates,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           rowRelDataType,
           grouping,
           namedProperties)
@@ -347,6 +360,8 @@ class DataSetWindowAggregate(
       isParserCaseSensitive: Boolean)
     : DataSet[Row] = {
 
+    val input = inputNode.asInstanceOf[DataSetRel]
+
     // create MapFunction for initializing the aggregations
     // it aligns the rowtime for pre-tumbling in case of a time-window for partial aggregates
     val mapFunction = createDataSetWindowPrepareMapFunction(
@@ -354,7 +369,8 @@ class DataSetWindowAggregate(
       window,
       namedAggregates,
       grouping,
-      inputType,
+      input.getRowType,
+      inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
       isParserCaseSensitive)
 
     val mappedDataSet = inputDS
@@ -390,7 +406,8 @@ class DataSetWindowAggregate(
           window,
           namedAggregates,
           grouping,
-          inputType,
+          input.getRowType,
+          inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
           isParserCaseSensitive)
 
         mappedDataSet.asInstanceOf[DataSet[Row]]
@@ -426,7 +443,8 @@ class DataSetWindowAggregate(
       generator,
       window,
       namedAggregates,
-      inputType,
+      input.getRowType,
+      inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes,
       rowRelDataType,
       grouping,
       namedProperties,

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
index c232a71..5697449 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
@@ -25,20 +25,18 @@ import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
 import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream}
 import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
 import org.apache.flink.table.api.StreamTableEnvironment
 import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
-import org.apache.flink.table.expressions._
+import org.apache.flink.table.expressions.ExpressionUtils._
 import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.plan.nodes.CommonAggregate
 import org.apache.flink.table.plan.nodes.datastream.DataStreamAggregate._
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.aggregate.AggregateUtil._
 import org.apache.flink.table.runtime.aggregate._
 import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
-import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
 import org.apache.flink.types.Row
 
 class DataStreamAggregate(
@@ -48,12 +46,12 @@ class DataStreamAggregate(
     traitSet: RelTraitSet,
     inputNode: RelNode,
     namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    rowRelDataType: RelDataType,
-    inputType: RelDataType,
+    schema: RowSchema,
+    inputSchema: RowSchema,
     grouping: Array[Int])
   extends SingleRel(cluster, traitSet, inputNode) with CommonAggregate with DataStreamRel {
 
-  override def deriveRowType(): RelDataType = rowRelDataType
+  override def deriveRowType(): RelDataType = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamAggregate(
@@ -63,22 +61,22 @@ class DataStreamAggregate(
       traitSet,
       inputs.get(0),
       namedAggregates,
-      getRowType,
-      inputType,
+      schema,
+      inputSchema,
       grouping)
   }
 
   override def toString: String = {
     s"Aggregate(${
       if (!grouping.isEmpty) {
-        s"groupBy: (${groupingToString(inputType, grouping)}), "
+        s"groupBy: (${groupingToString(inputSchema.logicalType, grouping)}), "
       } else {
         ""
       }
     }window: ($window), " +
       s"select: (${
         aggregationToString(
-          inputType,
+          inputSchema.logicalType,
           grouping,
           getRowType,
           namedAggregates,
@@ -88,13 +86,13 @@ class DataStreamAggregate(
 
   override def explainTerms(pw: RelWriter): RelWriter = {
     super.explainTerms(pw)
-      .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty)
+      .itemIf("groupBy", groupingToString(inputSchema.logicalType, grouping), !grouping.isEmpty)
       .item("window", window)
       .item(
         "select", aggregationToString(
-          inputType,
+          inputSchema.logicalType,
           grouping,
-          getRowType,
+          schema.logicalType,
           namedAggregates,
           namedProperties))
   }
@@ -102,17 +100,20 @@ class DataStreamAggregate(
   override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
 
     val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-
-    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
+    val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
+      new CalcitePair[AggregateCall, String](
+        inputSchema.mapAggregateCall(namedAggregate.left),
+        namedAggregate.right)
+    }
 
     val aggString = aggregationToString(
-      inputType,
+      inputSchema.logicalType,
       grouping,
-      getRowType,
+      schema.logicalType,
       namedAggregates,
       namedProperties)
 
-    val keyedAggOpName = s"groupBy: (${groupingToString(inputType, grouping)}), " +
+    val keyedAggOpName = s"groupBy: (${groupingToString(schema.logicalType, grouping)}), " +
       s"window: ($window), " +
       s"select: ($aggString)"
     val nonKeyedAggOpName = s"window: ($window), select: ($aggString)"
@@ -123,21 +124,21 @@ class DataStreamAggregate(
       inputDS.getType)
 
     val needMerge = window match {
-      case ProcessingTimeSessionGroupWindow(_, _) => true
-      case EventTimeSessionGroupWindow(_, _, _) => true
+      case SessionGroupWindow(_, _, _) => true
       case _ => false
     }
+    val physicalGrouping = grouping.map(inputSchema.mapIndex)
 
     // grouped / keyed aggregation
-    if (grouping.length > 0) {
+    if (physicalGrouping.length > 0) {
       val windowFunction = AggregateUtil.createAggregationGroupWindowFunction(
         window,
-        grouping.length,
-        namedAggregates.size,
-        rowRelDataType.getFieldCount,
+        physicalGrouping.length,
+        physicalNamedAggregates.size,
+        schema.physicalArity,
         namedProperties)
 
-      val keyedStream = inputDS.keyBy(grouping: _*)
+      val keyedStream = inputDS.keyBy(physicalGrouping: _*)
       val windowedStream =
         createKeyedWindowedStream(window, keyedStream)
           .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
@@ -145,20 +146,26 @@ class DataStreamAggregate(
       val (aggFunction, accumulatorRowType, aggResultRowType) =
         AggregateUtil.createDataStreamAggregateFunction(
           generator,
-          namedAggregates,
-          inputType,
-          rowRelDataType,
+          physicalNamedAggregates,
+          inputSchema.physicalType,
+          inputSchema.physicalFieldTypeInfo,
+          schema.physicalType,
           needMerge)
 
       windowedStream
-        .aggregate(aggFunction, windowFunction, accumulatorRowType, aggResultRowType, rowTypeInfo)
+        .aggregate(
+          aggFunction,
+          windowFunction,
+          accumulatorRowType,
+          aggResultRowType,
+          schema.physicalTypeInfo)
         .name(keyedAggOpName)
     }
     // global / non-keyed aggregation
     else {
       val windowFunction = AggregateUtil.createAggregationAllWindowFunction(
         window,
-        rowRelDataType.getFieldCount,
+        schema.physicalArity,
         namedProperties)
 
       val windowedStream =
@@ -168,13 +175,19 @@ class DataStreamAggregate(
       val (aggFunction, accumulatorRowType, aggResultRowType) =
         AggregateUtil.createDataStreamAggregateFunction(
           generator,
-          namedAggregates,
-          inputType,
-          rowRelDataType,
+          physicalNamedAggregates,
+          inputSchema.physicalType,
+          inputSchema.physicalFieldTypeInfo,
+          schema.physicalType,
           needMerge)
 
       windowedStream
-        .aggregate(aggFunction, windowFunction, accumulatorRowType, aggResultRowType, rowTypeInfo)
+        .aggregate(
+          aggFunction,
+          windowFunction,
+          accumulatorRowType,
+          aggResultRowType,
+          schema.physicalTypeInfo)
         .name(nonKeyedAggOpName)
     }
   }
@@ -186,95 +199,102 @@ object DataStreamAggregate {
   private def createKeyedWindowedStream(groupWindow: LogicalWindow, stream: KeyedStream[Row, Tuple])
     : WindowedStream[Row, Tuple, _ <: DataStreamWindow] = groupWindow match {
 
-    case ProcessingTimeTumblingGroupWindow(_, size) if isTimeInterval(size.resultType) =>
-      stream.window(TumblingProcessingTimeWindows.of(asTime(size)))
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
+      stream.window(TumblingProcessingTimeWindows.of(toTime(size)))
 
-    case ProcessingTimeTumblingGroupWindow(_, size) =>
-      stream.countWindow(asCount(size))
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
+      stream.countWindow(toLong(size))
 
-    case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
-      stream.window(TumblingEventTimeWindows.of(asTime(size)))
+    case TumblingGroupWindow(_, timeField, size)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
+      stream.window(TumblingEventTimeWindows.of(toTime(size)))
 
-    case EventTimeTumblingGroupWindow(_, _, size) =>
+    case TumblingGroupWindow(_, _, size) =>
       // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
       // before applying the  windowing logic. Otherwise, this would be the same as a
       // ProcessingTimeTumblingGroupWindow
       throw new UnsupportedOperationException(
         "Event-time grouping windows on row intervals are currently not supported.")
 
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) if isTimeInterval(size.resultType) =>
-      stream.window(SlidingProcessingTimeWindows.of(asTime(size), asTime(slide)))
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(slide) =>
+      stream.window(SlidingProcessingTimeWindows.of(toTime(size), toTime(slide)))
 
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) =>
-      stream.countWindow(asCount(size), asCount(slide))
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size) =>
+      stream.countWindow(toLong(size), toLong(slide))
 
-    case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
-      stream.window(SlidingEventTimeWindows.of(asTime(size), asTime(slide)))
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
+      stream.window(SlidingEventTimeWindows.of(toTime(size), toTime(slide)))
 
-    case EventTimeSlidingGroupWindow(_, _, size, slide) =>
+    case SlidingGroupWindow(_, _, size, slide) =>
       // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
       // before applying the  windowing logic. Otherwise, this would be the same as a
       // ProcessingTimeTumblingGroupWindow
       throw new UnsupportedOperationException(
         "Event-time grouping windows on row intervals are currently not supported.")
 
-    case ProcessingTimeSessionGroupWindow(_, gap: Expression) =>
-      stream.window(ProcessingTimeSessionWindows.withGap(asTime(gap)))
+    case SessionGroupWindow(_, timeField, gap)
+        if isProctimeAttribute(timeField) =>
+      stream.window(ProcessingTimeSessionWindows.withGap(toTime(gap)))
 
-    case EventTimeSessionGroupWindow(_, _, gap) =>
-      stream.window(EventTimeSessionWindows.withGap(asTime(gap)))
+    case SessionGroupWindow(_, timeField, gap)
+        if isRowtimeAttribute(timeField) =>
+      stream.window(EventTimeSessionWindows.withGap(toTime(gap)))
   }
 
   private def createNonKeyedWindowedStream(groupWindow: LogicalWindow, stream: DataStream[Row])
     : AllWindowedStream[Row, _ <: DataStreamWindow] = groupWindow match {
 
-    case ProcessingTimeTumblingGroupWindow(_, size) if isTimeInterval(size.resultType) =>
-      stream.windowAll(TumblingProcessingTimeWindows.of(asTime(size)))
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
+      stream.windowAll(TumblingProcessingTimeWindows.of(toTime(size)))
 
-    case ProcessingTimeTumblingGroupWindow(_, size) =>
-      stream.countWindowAll(asCount(size))
+    case TumblingGroupWindow(_, timeField, size)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
+      stream.countWindowAll(toLong(size))
 
-    case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
-      stream.windowAll(TumblingEventTimeWindows.of(asTime(size)))
+    case TumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
+      stream.windowAll(TumblingEventTimeWindows.of(toTime(size)))
 
-    case EventTimeTumblingGroupWindow(_, _, size) =>
+    case TumblingGroupWindow(_, _, size) =>
       // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
       // before applying the  windowing logic. Otherwise, this would be the same as a
       // ProcessingTimeTumblingGroupWindow
       throw new UnsupportedOperationException(
         "Event-time grouping windows on row intervals are currently not supported.")
 
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) if isTimeInterval(size.resultType) =>
-      stream.windowAll(SlidingProcessingTimeWindows.of(asTime(size), asTime(slide)))
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>
+      stream.windowAll(SlidingProcessingTimeWindows.of(toTime(size), toTime(slide)))
 
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) =>
-      stream.countWindowAll(asCount(size), asCount(slide))
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isProctimeAttribute(timeField) && isRowCountLiteral(size)=>
+      stream.countWindowAll(toLong(size), toLong(slide))
 
-    case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
-      stream.windowAll(SlidingEventTimeWindows.of(asTime(size), asTime(slide)))
+    case SlidingGroupWindow(_, timeField, size, slide)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
+      stream.windowAll(SlidingEventTimeWindows.of(toTime(size), toTime(slide)))
 
-    case EventTimeSlidingGroupWindow(_, _, size, slide) =>
+    case SlidingGroupWindow(_, _, size, slide) =>
       // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
       // before applying the  windowing logic. Otherwise, this would be the same as a
       // ProcessingTimeTumblingGroupWindow
       throw new UnsupportedOperationException(
         "Event-time grouping windows on row intervals are currently not supported.")
 
-    case ProcessingTimeSessionGroupWindow(_, gap) =>
-      stream.windowAll(ProcessingTimeSessionWindows.withGap(asTime(gap)))
+    case SessionGroupWindow(_, timeField, gap)
+        if isProctimeAttribute(timeField) && isTimeIntervalLiteral(gap) =>
+      stream.windowAll(ProcessingTimeSessionWindows.withGap(toTime(gap)))
 
-    case EventTimeSessionGroupWindow(_, _, gap) =>
-      stream.windowAll(EventTimeSessionWindows.withGap(asTime(gap)))
+    case SessionGroupWindow(_, timeField, gap)
+        if isRowtimeAttribute(timeField) && isTimeIntervalLiteral(gap) =>
+      stream.windowAll(EventTimeSessionWindows.withGap(toTime(gap)))
   }
 
-  def asTime(expr: Expression): Time = expr match {
-    case Literal(value: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) => Time.milliseconds(value)
-    case _ => throw new IllegalArgumentException()
-  }
 
-  def asCount(expr: Expression): Long = expr match {
-    case Literal(value: Long, RowIntervalTypeInfo.INTERVAL_ROWS) => value
-    case _ => throw new IllegalArgumentException()
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
index b015a1d..c6c25c0 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
@@ -27,9 +27,9 @@ import org.apache.calcite.rex.RexProgram
 import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
-import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.nodes.CommonCalc
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.types.Row
 
 /**
@@ -40,17 +40,25 @@ class DataStreamCalc(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
     input: RelNode,
-    rowRelDataType: RelDataType,
+    inputSchema: RowSchema,
+    schema: RowSchema,
     calcProgram: RexProgram,
     ruleDescription: String)
   extends Calc(cluster, traitSet, input, calcProgram)
   with CommonCalc
   with DataStreamRel {
 
-  override def deriveRowType(): RelDataType = rowRelDataType
+  override def deriveRowType(): RelDataType = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = {
-    new DataStreamCalc(cluster, traitSet, child, getRowType, program, ruleDescription)
+    new DataStreamCalc(
+      cluster,
+      traitSet,
+      child,
+      inputSchema,
+      schema,
+      program,
+      ruleDescription)
   }
 
   override def toString: String = calcToString(calcProgram, getExpressionString)
@@ -85,8 +93,8 @@ class DataStreamCalc(
 
     val body = functionBody(
       generator,
-      inputDataStream.getType,
-      getRowType,
+      inputSchema,
+      schema,
       calcProgram,
       config)
 
@@ -94,7 +102,7 @@ class DataStreamCalc(
       ruleDescription,
       classOf[FlatMapFunction[Row, Row]],
       body,
-      FlinkTypeFactory.toInternalRowTypeInfo(getRowType))
+      schema.physicalTypeInfo)
 
     val mapFunc = calcMapFunction(genFunction)
     inputDataStream.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
index 342920a..8955110 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
@@ -18,7 +18,6 @@
 package org.apache.flink.table.plan.nodes.datastream
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
 import org.apache.calcite.rex.{RexCall, RexNode}
 import org.apache.calcite.sql.SemiJoinType
@@ -28,6 +27,7 @@ import org.apache.flink.table.api.StreamTableEnvironment
 import org.apache.flink.table.functions.utils.TableSqlFunction
 import org.apache.flink.table.plan.nodes.CommonCorrelate
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.types.Row
 
 /**
@@ -36,28 +36,30 @@ import org.apache.flink.types.Row
 class DataStreamCorrelate(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
+    inputSchema: RowSchema,
     inputNode: RelNode,
     scan: FlinkLogicalTableFunctionScan,
     condition: Option[RexNode],
-    relRowType: RelDataType,
-    joinRowType: RelDataType,
+    schema: RowSchema,
+    joinSchema: RowSchema,
     joinType: SemiJoinType,
     ruleDescription: String)
   extends SingleRel(cluster, traitSet, inputNode)
   with CommonCorrelate
   with DataStreamRel {
 
-  override def deriveRowType() = relRowType
+  override def deriveRowType() = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamCorrelate(
       cluster,
       traitSet,
+      inputSchema,
       inputs.get(0),
       scan,
       condition,
-      relRowType,
-      joinRowType,
+      schema,
+      joinSchema,
       joinType,
       ruleDescription)
   }
@@ -74,7 +76,7 @@ class DataStreamCorrelate(
     super.explainTerms(pw)
       .item("invocation", scan.getCall)
       .item("function", sqlFunction.getTableFunction.getClass.getCanonicalName)
-      .item("rowType", relRowType)
+      .item("rowType", schema.logicalType)
       .item("joinType", joinType)
       .itemIf("condition", condition.orNull, condition.isDefined)
   }
@@ -94,16 +96,16 @@ class DataStreamCorrelate(
 
     val mapFunc = correlateMapFunction(
       config,
-      inputDS.getType,
+      inputSchema,
       udtfTypeInfo,
-      getRowType,
+      schema,
       joinType,
       rexCall,
       condition,
       Some(pojoFieldMapping),
       ruleDescription)
 
-    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, relRowType))
+    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, schema.logicalType))
   }
 
 }


[02/15] flink git commit: [FLINK-5884] [table] Integrate time indicators for Table API & SQL

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
index 63dc1ae..31ad558 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
@@ -24,7 +24,7 @@ import org.apache.calcite.sql.{SqlFunction, SqlOperator, SqlOperatorTable}
 import org.apache.flink.table.api.ValidationException
 import org.apache.flink.table.expressions._
 import org.apache.flink.table.functions.utils.{AggSqlFunction, ScalarSqlFunction, TableSqlFunction}
-import org.apache.flink.table.functions.{AggregateFunction, EventTimeExtractor, RowTime, ScalarFunction, TableFunction, _}
+import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction}
 
 import scala.collection.JavaConversions._
 import scala.collection.mutable
@@ -242,15 +242,11 @@ object FunctionCatalog {
     // array
     "cardinality" -> classOf[ArrayCardinality],
     "at" -> classOf[ArrayElementAt],
-    "element" -> classOf[ArrayElement],
+    "element" -> classOf[ArrayElement]
 
     // TODO implement function overloading here
     // "floor" -> classOf[TemporalFloor]
     // "ceil" -> classOf[TemporalCeil]
-
-    // extensions to support streaming query
-    "rowtime" -> classOf[RowTime],
-    "proctime" -> classOf[ProcTime]
   )
 
   /**
@@ -392,8 +388,6 @@ class BasicOperatorTable extends ReflectiveSqlOperatorTable {
     SqlStdOperatorTable.ROUND,
     SqlStdOperatorTable.PI,
     // EXTENSIONS
-    EventTimeExtractor,
-    ProcTimeExtractor,
     SqlStdOperatorTable.TUMBLE,
     SqlStdOperatorTable.TUMBLE_START,
     SqlStdOperatorTable.TUMBLE_END,

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
index cab3855..81c60b4 100644
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
@@ -406,15 +406,6 @@ public class TableEnvironmentITCase extends TableProgramsCollectionTestBase {
 	}
 
 	@Test(expected = TableException.class)
-	public void testAsWithToFewFields() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. Not enough field names specified.
-		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b");
-	}
-
-	@Test(expected = TableException.class)
 	public void testAsWithToManyFields() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
index 9939a9c..faacc54 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
@@ -93,7 +93,8 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2"),
         UnresolvedFieldReference("name3")
-    ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -107,7 +108,8 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2"),
         UnresolvedFieldReference("name3")
-    ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -121,7 +123,8 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2"),
         UnresolvedFieldReference("name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
   }
 
   @Test
@@ -132,7 +135,8 @@ class TableEnvironmentTest extends TableTestBase {
         UnresolvedFieldReference("pf3"),
         UnresolvedFieldReference("pf1"),
         UnresolvedFieldReference("pf2")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("pf3", "pf1", "pf2")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -142,7 +146,8 @@ class TableEnvironmentTest extends TableTestBase {
   def testGetFieldInfoAtomicName1(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(
       atomicType,
-      Array(UnresolvedFieldReference("name"))
+      Array(UnresolvedFieldReference("name")),
+      ignoreTimeAttributes = true
     )
 
     fieldInfo._1.zip(Array("name")).foreach(x => assertEquals(x._2, x._1))
@@ -156,7 +161,8 @@ class TableEnvironmentTest extends TableTestBase {
       Array(
         UnresolvedFieldReference("name1"),
         UnresolvedFieldReference("name2")
-      ))
+      ),
+      ignoreTimeAttributes = true)
   }
 
   @Test
@@ -167,7 +173,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("f0"), "name1"),
         Alias(UnresolvedFieldReference("f1"), "name2"),
         Alias(UnresolvedFieldReference("f2"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -181,7 +188,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("f2"), "name1"),
         Alias(UnresolvedFieldReference("f0"), "name2"),
         Alias(UnresolvedFieldReference("f1"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -195,7 +203,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("xxx"), "name1"),
         Alias(UnresolvedFieldReference("yyy"), "name2"),
         Alias(UnresolvedFieldReference("zzz"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
   }
 
   @Test
@@ -206,7 +215,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("cf1"), "name1"),
         Alias(UnresolvedFieldReference("cf2"), "name2"),
         Alias(UnresolvedFieldReference("cf3"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -220,7 +230,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("cf3"), "name1"),
         Alias(UnresolvedFieldReference("cf1"), "name2"),
         Alias(UnresolvedFieldReference("cf2"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -234,7 +245,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("xxx"), "name1"),
         Alias(UnresolvedFieldReference("yyy"), "name2"),
         Alias(UnresolvedFieldReference("zzz"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
   }
 
   @Test
@@ -245,7 +257,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("pf1"), "name1"),
         Alias(UnresolvedFieldReference("pf2"), "name2"),
         Alias(UnresolvedFieldReference("pf3"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
@@ -259,7 +272,8 @@ class TableEnvironmentTest extends TableTestBase {
         Alias(UnresolvedFieldReference("pf3"), "name1"),
         Alias(UnresolvedFieldReference("pf1"), "name2"),
         Alias(UnresolvedFieldReference("pf2"), "name3")
-      ))
+      ),
+      ignoreTimeAttributes = true)
 
     fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
     fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
@@ -272,8 +286,9 @@ class TableEnvironmentTest extends TableTestBase {
       Array(
         Alias(UnresolvedFieldReference("xxx"), "name1"),
         Alias(UnresolvedFieldReference("yyy"), "name2"),
-        Alias( UnresolvedFieldReference("zzz"), "name3")
-      ))
+        Alias(UnresolvedFieldReference("zzz"), "name3")
+      ),
+      ignoreTimeAttributes = true)
   }
 
   @Test(expected = classOf[TableException])
@@ -282,12 +297,16 @@ class TableEnvironmentTest extends TableTestBase {
       atomicType,
       Array(
         Alias(UnresolvedFieldReference("name1"), "name2")
-      ))
+      ),
+      ignoreTimeAttributes = true)
   }
 
   @Test(expected = classOf[TableException])
   def testGetFieldInfoGenericRowAlias(): Unit = {
-    tEnv.getFieldInfo(genericRowType, Array(UnresolvedFieldReference("first")))
+    tEnv.getFieldInfo(
+      genericRowType,
+      Array(UnresolvedFieldReference("first")),
+      ignoreTimeAttributes = true)
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
index e61e190..57ee3b3 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
@@ -208,16 +208,6 @@ class TableEnvironmentITCase(
   }
 
   @Test(expected = classOf[TableException])
-  def testToTableWithToFewFields(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env)
-      // Must fail. Number of fields does not match.
-      .toTable(tEnv, 'a, 'b)
-  }
-
-  @Test(expected = classOf[TableException])
   def testToTableWithToManyFields(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env, config)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/WindowAggregateTest.scala
index 0ccb557..71d0002 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/WindowAggregateTest.scala
@@ -47,7 +47,7 @@ class WindowAggregateTest extends TableTestBase {
           batchTableNode(0),
           term("select", "ts, a, b")
         ),
-        term("window", EventTimeTumblingGroupWindow('w$, 'ts, 7200000.millis)),
+        term("window", TumblingGroupWindow('w$, 'ts, 7200000.millis)),
         term("select", "SUM(a) AS sumA, COUNT(b) AS cntB")
       )
 
@@ -76,7 +76,7 @@ class WindowAggregateTest extends TableTestBase {
           "DataSetWindowAggregate",
           batchTableNode(0),
           term("groupBy", "c"),
-          term("window", EventTimeTumblingGroupWindow('w$, 'ts, 240000.millis)),
+          term("window", TumblingGroupWindow('w$, 'ts, 240000.millis)),
           term("select", "c, SUM(a) AS sumA, MIN(b) AS minB, " +
             "start('w$) AS w$start, end('w$) AS w$end")
         ),
@@ -106,7 +106,7 @@ class WindowAggregateTest extends TableTestBase {
           batchTableNode(0),
           term("select", "ts, b, a")
         ),
-        term("window", EventTimeTumblingGroupWindow('w$, 'ts, 240000.millis)),
+        term("window", TumblingGroupWindow('w$, 'ts, 240000.millis)),
         term("select", "weightedAvg(b, a) AS wAvg")
       )
 
@@ -132,7 +132,7 @@ class WindowAggregateTest extends TableTestBase {
           term("select", "ts, a, b")
         ),
         term("window",
-          EventTimeSlidingGroupWindow('w$, 'ts, 5400000.millis, 900000.millis)),
+          SlidingGroupWindow('w$, 'ts, 5400000.millis, 900000.millis)),
         term("select", "SUM(a) AS sumA, COUNT(b) AS cntB")
       )
 
@@ -162,7 +162,7 @@ class WindowAggregateTest extends TableTestBase {
           batchTableNode(0),
           term("groupBy", "c, d"),
           term("window",
-            EventTimeSlidingGroupWindow('w$, 'ts, 10800000.millis, 3600000.millis)),
+            SlidingGroupWindow('w$, 'ts, 10800000.millis, 3600000.millis)),
           term("select", "c, d, SUM(a) AS sumA, AVG(b) AS avgB, " +
             "start('w$) AS w$start, end('w$) AS w$end")
         ),
@@ -188,7 +188,7 @@ class WindowAggregateTest extends TableTestBase {
           batchTableNode(0),
           term("select", "ts")
         ),
-        term("window", EventTimeSessionGroupWindow('w$, 'ts, 1800000.millis)),
+        term("window", SessionGroupWindow('w$, 'ts, 1800000.millis)),
         term("select", "COUNT(*) AS cnt")
       )
 
@@ -217,7 +217,7 @@ class WindowAggregateTest extends TableTestBase {
           "DataSetWindowAggregate",
           batchTableNode(0),
           term("groupBy", "c, d"),
-          term("window", EventTimeSessionGroupWindow('w$, 'ts, 43200000.millis)),
+          term("window", SessionGroupWindow('w$, 'ts, 43200000.millis)),
           term("select", "c, d, SUM(a) AS sumA, MIN(b) AS minB, " +
             "start('w$) AS w$start, end('w$) AS w$end")
         ),
@@ -249,7 +249,7 @@ class WindowAggregateTest extends TableTestBase {
             term("select", "ts, c")
           ),
           term("groupBy", "c"),
-          term("window", EventTimeTumblingGroupWindow('w$, 'ts, 240000.millis)),
+          term("window", TumblingGroupWindow('w$, 'ts, 240000.millis)),
           term("select", "c, start('w$) AS w$start, end('w$) AS w$end")
         ),
         term("select", "CAST(w$end) AS w$end")
@@ -304,7 +304,7 @@ class WindowAggregateTest extends TableTestBase {
 
     val sql = "SELECT COUNT(*) " +
       "FROM T " +
-      "GROUP BY TUMBLE(proctime(), b * INTERVAL '1' MINUTE)"
+      "GROUP BY TUMBLE(ts, b * INTERVAL '1' MINUTE)"
     util.verifySql(sql, "n/a")
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
index 6ebfec0..b484293 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
@@ -19,14 +19,12 @@ package org.apache.flink.table.api.scala.batch.table
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.ValidationException
-import org.apache.flink.table.expressions.{RowtimeAttribute, Upper, WindowReference}
-import org.apache.flink.table.functions.ScalarFunction
 import org.apache.flink.table.api.scala.batch.table.FieldProjectionTest._
-import org.apache.flink.table.plan.logical.EventTimeTumblingGroupWindow
-import org.apache.flink.table.utils._
-import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.expressions.{Upper, WindowReference}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.plan.logical.TumblingGroupWindow
 import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{TableTestBase, _}
 import org.junit.Test
 
 /**
@@ -223,7 +221,8 @@ class FieldProjectionTest extends TableTestBase {
 
   @Test
   def testSelectFromStreamingWindow(): Unit = {
-    val sourceTable = streamUtil.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val sourceTable = streamUtil
+      .addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd, 'rowtime.rowtime)
     val resultTable = sourceTable
         .window(Tumble over 5.millis on 'rowtime as 'w)
         .groupBy('w)
@@ -235,14 +234,14 @@ class FieldProjectionTest extends TableTestBase {
         unaryNode(
           "DataStreamCalc",
           streamTableNode(0),
-          term("select", "c", "a", "UPPER(c) AS $f2")
+          term("select", "c", "a", "rowtime", "UPPER(c) AS $f3")
         ),
         term("window",
-          EventTimeTumblingGroupWindow(
-           WindowReference("w"),
-            RowtimeAttribute(),
+          TumblingGroupWindow(
+            WindowReference("w"),
+            'rowtime,
             5.millis)),
-        term("select", "COUNT($f2) AS TMP_0", "SUM(a) AS TMP_1")
+        term("select", "COUNT($f3) AS TMP_0", "SUM(a) AS TMP_1")
       )
 
     streamUtil.verifyTable(resultTable, expected)
@@ -250,7 +249,8 @@ class FieldProjectionTest extends TableTestBase {
 
   @Test
   def testSelectFromStreamingGroupedWindow(): Unit = {
-    val sourceTable = streamUtil.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val sourceTable = streamUtil
+      .addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd, 'rowtime.rowtime)
     val resultTable = sourceTable
         .window(Tumble over 5.millis on 'rowtime as 'w)
         .groupBy('w, 'b)
@@ -263,15 +263,15 @@ class FieldProjectionTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "c", "a", "b", "UPPER(c) AS $f3")
+            term("select", "c", "a", "b", "rowtime", "UPPER(c) AS $f4")
           ),
           term("groupBy", "b"),
           term("window",
-            EventTimeTumblingGroupWindow(
-             WindowReference("w"),
-              RowtimeAttribute(),
+            TumblingGroupWindow(
+              WindowReference("w"),
+              'rowtime,
               5.millis)),
-          term("select", "b", "COUNT($f3) AS TMP_0", "SUM(a) AS TMP_1")
+          term("select", "b", "COUNT($f4) AS TMP_0", "SUM(a) AS TMP_1")
         ),
         term("select", "TMP_0", "TMP_1", "b")
     )

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
index 8a20f6d..c481105 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/GroupWindowTest.scala
@@ -63,25 +63,24 @@ class GroupWindowTest extends TableTestBase {
   //===============================================================================================
 
   @Test(expected = classOf[ValidationException])
-  def testProcessingTimeTumblingGroupWindowOverTime(): Unit = {
+  def testInvalidProcessingTimeDefinition(): Unit = {
     val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .window(Tumble over 50.milli as 'w)   // require a time attribute
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
+    // proctime is not allowed
+    util.addTable[(Long, Int, String)]('long.proctime, 'int, 'string)
   }
 
   @Test(expected = classOf[ValidationException])
-  def testProcessingTimeTumblingGroupWindowOverCount(): Unit = {
+  def testInvalidProcessingTimeDefinition2(): Unit = {
     val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    // proctime is not allowed
+    util.addTable[(Long, Int, String)]('long, 'int, 'string, 'proctime.proctime)
+  }
 
-    table
-      .window(Tumble over 2.rows as 'w)   // require a time attribute
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
+  @Test(expected = classOf[ValidationException])
+  def testInvalidEventTimeDefinition(): Unit = {
+    val util = batchTestUtil()
+    // definition must not extend schema
+    util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -101,7 +100,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeTumblingGroupWindowOverCount(): Unit = {
     val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
       .window(Tumble over 2.rows on 'long as 'w)
@@ -112,7 +111,7 @@ class GroupWindowTest extends TableTestBase {
       "DataSetWindowAggregate",
       batchTableNode(0),
       term("groupBy", "string"),
-      term("window", EventTimeTumblingGroupWindow(WindowReference("w"), 'long, 2.rows)),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'long, 2.rows)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -135,7 +134,7 @@ class GroupWindowTest extends TableTestBase {
       "DataSetWindowAggregate",
       batchTableNode(0),
       term("groupBy", "string"),
-      term("window", EventTimeTumblingGroupWindow(WindowReference("w"), 'long, 5.milli)),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'long, 5.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -145,7 +144,7 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeTumblingGroupWindowOverTime(): Unit = {
     val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
       .window(Tumble over 5.milli on 'long as 'w)
@@ -156,7 +155,7 @@ class GroupWindowTest extends TableTestBase {
       "DataSetWindowAggregate",
       batchTableNode(0),
       term("groupBy", "string"),
-      term("window", EventTimeTumblingGroupWindow(WindowReference("w"), 'long, 5.milli)),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'long, 5.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -164,28 +163,6 @@ class GroupWindowTest extends TableTestBase {
   }
 
   @Test(expected = classOf[ValidationException])
-  def testAllProcessingTimeTumblingGroupWindowOverTime(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .window(Tumble over 50.milli as 'w) // require a time attribute
-      .groupBy('w)
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .window(Tumble over 2.rows as 'w) // require a time attribute
-      .groupBy('w)
-      .select('int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
   def testAllTumblingGroupWindowWithInvalidUdAggArgs(): Unit = {
     val util = batchTestUtil()
     val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
@@ -216,7 +193,7 @@ class GroupWindowTest extends TableTestBase {
         batchTableNode(0),
         term("select", "int", "long")
       ),
-      term("window", EventTimeTumblingGroupWindow(WindowReference("w"), 'long, 5.milli)),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'long, 5.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -240,7 +217,7 @@ class GroupWindowTest extends TableTestBase {
         batchTableNode(0),
         term("select", "int", "long")
       ),
-      term("window", EventTimeTumblingGroupWindow(WindowReference("w"), 'long, 2.rows)),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'long, 2.rows)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -252,28 +229,6 @@ class GroupWindowTest extends TableTestBase {
   //===============================================================================================
 
   @Test(expected = classOf[ValidationException])
-  def testProcessingTimeSlidingGroupWindowOverTime(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .window(Slide over 50.milli every 50.milli as 'w) // require on a time attribute
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .window(Slide over 10.rows every 5.rows as 'w) // require on a time attribute
-      .groupBy('w, 'string)
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
   def testSlidingGroupWindowWithInvalidUdAggArgs(): Unit = {
     val util = batchTestUtil()
     val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
@@ -302,7 +257,7 @@ class GroupWindowTest extends TableTestBase {
       batchTableNode(0),
       term("groupBy", "string"),
       term("window",
-        EventTimeSlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
+        SlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -324,7 +279,7 @@ class GroupWindowTest extends TableTestBase {
       batchTableNode(0),
       term("groupBy", "string"),
       term("window",
-        EventTimeSlidingGroupWindow(WindowReference("w"), 'long, 2.rows, 1.rows)),
+        SlidingGroupWindow(WindowReference("w"), 'long, 2.rows, 1.rows)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -348,7 +303,7 @@ class GroupWindowTest extends TableTestBase {
       batchTableNode(0),
       term("groupBy", "string"),
       term("window",
-           EventTimeSlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
+           SlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -356,17 +311,6 @@ class GroupWindowTest extends TableTestBase {
   }
 
   @Test(expected = classOf[ValidationException])
-  def testAllProcessingTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .window(Slide over 2.rows every 1.rows as 'w) // require on a time attribute
-      .groupBy('w)
-      .select('int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
   def testAllSlidingGroupWindowWithInvalidUdAggArgs(): Unit = {
     val util = batchTestUtil()
     val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
@@ -374,7 +318,7 @@ class GroupWindowTest extends TableTestBase {
     val myWeightedAvg = new WeightedAvgWithMerge
 
     table
-      .window(Slide over 2.minutes every 1.minute on 'rowtime as 'w)
+      .window(Slide over 2.minutes every 1.minute on 'long as 'w)
       .groupBy('w)
       // invalid function arguments
       .select(myWeightedAvg('int, 'string))
@@ -398,7 +342,7 @@ class GroupWindowTest extends TableTestBase {
         term("select", "int", "long")
       ),
       term("window",
-        EventTimeSlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
+        SlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -423,7 +367,7 @@ class GroupWindowTest extends TableTestBase {
         term("select", "int", "long")
       ),
       term("window",
-        EventTimeSlidingGroupWindow(WindowReference("w"), 'long, 2.rows, 1.rows)),
+        SlidingGroupWindow(WindowReference("w"), 'long, 2.rows, 1.rows)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -448,7 +392,7 @@ class GroupWindowTest extends TableTestBase {
       "DataSetWindowAggregate",
       batchTableNode(0),
       term("groupBy", "string"),
-      term("window", EventTimeSessionGroupWindow(WindowReference("w"), 'long, 7.milli)),
+      term("window", SessionGroupWindow(WindowReference("w"), 'long, 7.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -471,7 +415,7 @@ class GroupWindowTest extends TableTestBase {
       "DataSetWindowAggregate",
       batchTableNode(0),
       term("groupBy", "string"),
-      term("window", EventTimeSessionGroupWindow(WindowReference("w"), 'long, 7.milli)),
+      term("window", SessionGroupWindow(WindowReference("w"), 'long, 7.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -479,17 +423,6 @@ class GroupWindowTest extends TableTestBase {
   }
 
   @Test(expected = classOf[ValidationException])
-  def testProcessingTimeSessionGroupWindowOverTime(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Session withGap 7.milli as 'w) // require on a time attribute
-      .groupBy('string, 'w)
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
   def testSessionGroupWindowWithInvalidUdAggArgs(): Unit = {
     val util = batchTestUtil()
     val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index 67d13b0..6bab4b3 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -197,14 +197,14 @@ class SqlITCase extends StreamingWithStateTestBase {
     // for sum aggregation ensure that every time the order of each element is consistent
     env.setParallelism(1)
 
-    val t1 = env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c)
+    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY ProcTime()  RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (PARTITION BY c ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (PARTITION BY c ORDER BY proctime  RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding) as cnt2 " +
       "from T1"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -224,13 +224,13 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t1 = env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c)
+    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY ProcTime() ROWS BETWEEN UNBOUNDED preceding AND " +
+      "count(a) OVER (PARTITION BY c ORDER BY proctime ROWS BETWEEN UNBOUNDED preceding AND " +
       "CURRENT ROW)" +
       "from T1"
 
@@ -254,14 +254,14 @@ class SqlITCase extends StreamingWithStateTestBase {
     // for sum aggregation ensure that every time the order of each element is consistent
     env.setParallelism(1)
 
-    val t1 = env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c)
+    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, " +
-      "count(a) OVER (ORDER BY ProcTime()  RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (ORDER BY proctime  RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (ORDER BY proctime RANGE UNBOUNDED preceding) as cnt2 " +
       "from T1"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -281,12 +281,12 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t1 = env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c)
+    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
-      "count(a) OVER (ORDER BY ProcTime() ROWS BETWEEN UNBOUNDED preceding AND CURRENT ROW)" +
+      "count(a) OVER (ORDER BY proctime ROWS BETWEEN UNBOUNDED preceding AND CURRENT ROW)" +
       "from T1"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -328,14 +328,14 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val t1 = env
       .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, a, " +
-      "count(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
-      ", sum(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
+      "count(a) OVER (PARTITION BY c ORDER BY rowtime ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
+      ", sum(a) OVER (PARTITION BY c ORDER BY rowtime ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
       " from T1"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -385,14 +385,14 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val t1 = env
       .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, a, " +
-      "count(a) OVER (ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)," +
-      "sum(a) OVER (ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
+      "count(a) OVER (ORDER BY rowtime ROWS BETWEEN 2 preceding AND CURRENT ROW)," +
+      "sum(a) OVER (ORDER BY rowtime ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
       "from T1"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -453,15 +453,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val t1 = env
       .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, b, " +
-      "count(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      "count(a) OVER (PARTITION BY c ORDER BY rowtime RANGE BETWEEN INTERVAL '1' SECOND " +
       "preceding AND CURRENT ROW)" +
-      ", sum(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      ", sum(a) OVER (PARTITION BY c ORDER BY rowtime RANGE BETWEEN INTERVAL '1' SECOND " +
       " preceding AND CURRENT ROW)" +
       " from T1"
 
@@ -525,15 +525,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val t1 = env
       .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, b, " +
-      "count(a) OVER (ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      "count(a) OVER (ORDER BY rowtime RANGE BETWEEN INTERVAL '1' SECOND " +
       "preceding AND CURRENT ROW)" +
-      ", sum(a) OVER (ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      ", sum(a) OVER (ORDER BY rowtime RANGE BETWEEN INTERVAL '1' SECOND " +
       " preceding AND CURRENT ROW)" +
       " from T1"
 
@@ -565,14 +565,14 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t1 = env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c)
+    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
     tEnv.registerTable("T1", t1)
 
     val sqlQuery = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (PARTITION BY b ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (PARTITION BY b ORDER BY proctime RANGE UNBOUNDED preceding) as cnt2 " +
       "from T1"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -592,15 +592,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val sqlQuery = "SELECT a, b, c, " +
       "SUM(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "count(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "avg(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "max(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "min(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row) " +
+      "partition by a order by rowtime rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -632,7 +632,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     )
 
     val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
@@ -670,15 +670,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val sqlQuery = "SELECT a, b, c, " +
       "SUM(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "count(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "avg(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "max(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
+      "partition by a order by rowtime rows between unbounded preceding and current row), " +
       "min(b) over (" +
-      "partition by a order by rowtime() rows between unbounded preceding and current row) " +
+      "partition by a order by rowtime rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -702,7 +702,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     )
 
     val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
@@ -740,11 +740,11 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
-      "SUM(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "count(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "avg(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "max(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "min(b) over (order by rowtime() rows between unbounded preceding and current row) " +
+      "SUM(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -764,7 +764,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     )
 
     val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
@@ -795,11 +795,11 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
-      "SUM(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "count(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "avg(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "max(b) over (order by rowtime() rows between unbounded preceding and current row), " +
-      "min(b) over (order by rowtime() rows between unbounded preceding and current row) " +
+      "SUM(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime rows between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -820,7 +820,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     )
 
     val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
@@ -852,11 +852,11 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
-      "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
+      "SUM(b) over (order by rowtime range between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime range between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime range between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime range between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime range between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -878,7 +878,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     )
 
     val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
@@ -916,15 +916,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val sqlQuery = "SELECT a, b, c, " +
       "SUM(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime range between unbounded preceding and current row), " +
       "count(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime range between unbounded preceding and current row), " +
       "avg(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime range between unbounded preceding and current row), " +
       "max(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime range between unbounded preceding and current row), " +
       "min(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row) " +
+      "partition by a order by rowtime range between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -946,7 +946,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     )
 
     val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
-      .toTable(tEnv).as('a, 'b, 'c)
+      .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
 
     tEnv.registerTable("T1", t1)
 
@@ -981,14 +981,15 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    val t = StreamTestData.get5TupleDataStream(env)
+      .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
     tEnv.registerTable("MyTable", t)
 
     val sqlQuery = "SELECT a,  " +
       " SUM(c) OVER (" +
-      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " PARTITION BY a ORDER BY proctime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC , " +
       " MIN(c) OVER (" +
-      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC " +
+      " PARTITION BY a ORDER BY proctime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC " +
       " FROM MyTable"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -1023,14 +1024,15 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    val t = StreamTestData.get5TupleDataStream(env)
+      .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
     tEnv.registerTable("MyTable", t)
 
     val sqlQuery = "SELECT a,  " +
       " SUM(c) OVER (" +
-      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " PARTITION BY a ORDER BY proctime ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) AS sumC , " +
       " MIN(c) OVER (" +
-      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) AS minC " +
+      " PARTITION BY a ORDER BY proctime ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) AS minC " +
       " FROM MyTable"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -1066,14 +1068,15 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    val t = StreamTestData.get5TupleDataStream(env)
+      .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
     tEnv.registerTable("MyTable", t)
 
     val sqlQuery = "SELECT a,  " +
       " SUM(c) OVER (" +
-      " ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " ORDER BY proctime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC , " +
       " MIN(c) OVER (" +
-      " ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC " +
+      " ORDER BY proctime ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC " +
       " FROM MyTable"
 
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
@@ -1108,14 +1111,15 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
     StreamITCase.testResults = mutable.MutableList()
 
-    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    val t = StreamTestData.get5TupleDataStream(env)
+      .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
     tEnv.registerTable("MyTable", t)
 
     val sqlQuery = "SELECT a,  " +
       " SUM(c) OVER (" +
-      " ORDER BY procTime() ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " ORDER BY proctime ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) AS sumC , " +
       " MIN(c) OVER (" +
-      " ORDER BY procTime() ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) AS minC " +
+      " ORDER BY proctime ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) AS minC " +
       " FROM MyTable"
     val result = tEnv.sql(sqlQuery).toDataStream[Row]
     result.addSink(new StreamITCase.StringSink)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index cef2665..edf7b1d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -21,22 +21,23 @@ import org.apache.flink.api.scala._
 import org.apache.flink.table.api.{TableException, ValidationException}
 import org.apache.flink.table.api.java.utils.UserDefinedAggFunctions.WeightedAvgWithMerge
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.plan.logical.{EventTimeTumblingGroupWindow, ProcessingTimeSessionGroupWindow, ProcessingTimeSlidingGroupWindow}
+import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.utils.TableTestUtil._
 import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
-import org.junit.Test
+import org.junit.{Ignore, Test}
 
 class WindowAggregateTest extends TableTestBase {
   private val streamUtil: StreamTableTestUtil = streamTestUtil()
-  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c)
+  streamUtil.addTable[(Int, String, Long)](
+    "MyTable", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
 
   @Test
   def testNonPartitionedProcessingTimeBoundedWindow() = {
 
-    val sqlQuery = "SELECT a, Count(c) OVER (ORDER BY procTime()" +
-      "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS countA FROM MyTable"
-
-    val expected =
+    val sqlQuery = "SELECT a, Count(c) OVER (ORDER BY proctime  " +
+      "RANGE BETWEEN INTERVAL '10' SECOND PRECEDING AND CURRENT ROW) AS countA " +
+      "FROM MyTable"
+      val expected =
       unaryNode(
         "DataStreamCalc",
         unaryNode(
@@ -44,11 +45,11 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN 10000 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(c) AS w0$o0")
+          term("select", "a", "c", "proctime", "COUNT(c) AS w0$o0")
         ),
         term("select", "a", "w0$o0 AS $1")
       )
@@ -59,7 +60,7 @@ class WindowAggregateTest extends TableTestBase {
   @Test
   def testPartitionedProcessingTimeBoundedWindow() = {
 
-    val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY procTime()" +
+    val sqlQuery = "SELECT a, AVG(c) OVER (PARTITION BY a ORDER BY proctime " +
       "RANGE BETWEEN INTERVAL '2' HOUR PRECEDING AND CURRENT ROW) AS avgA " +
       "FROM MyTable"
       val expected =
@@ -70,12 +71,12 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
           term("partitionBy","a"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN 7200000 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(c) AS w0$o0", "$SUM0(c) AS w0$o1")
+          term("select", "a", "c", "proctime", "COUNT(c) AS w0$o0", "$SUM0(c) AS w0$o1")
         ),
         term("select", "a", "/(CASE(>(w0$o0, 0)", "CAST(w0$o1), null), w0$o0) AS avgA")
       )
@@ -84,23 +85,24 @@ class WindowAggregateTest extends TableTestBase {
   }
 
   @Test
+  @Ignore // TODO enable once CALCITE-1761 is fixed
   def testTumbleFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
     val sql =
       "SELECT " +
         "  COUNT(*), weightedAvg(c, a) AS wAvg, " +
-        "  TUMBLE_START(rowtime(), INTERVAL '15' MINUTE), " +
-        "  TUMBLE_END(rowtime(), INTERVAL '15' MINUTE)" +
+        "  TUMBLE_START(rowtime, INTERVAL '15' MINUTE), " +
+        "  TUMBLE_END(rowtime, INTERVAL '15' MINUTE)" +
         "FROM MyTable " +
-        "GROUP BY TUMBLE(rowtime(), INTERVAL '15' MINUTE)"
+        "GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE)"
     val expected =
       unaryNode(
         "DataStreamCalc",
         unaryNode(
           "DataStreamAggregate",
           streamTableNode(0),
-          term("window", EventTimeTumblingGroupWindow('w$, 'rowtime, 900000.millis)),
+          term("window", TumblingGroupWindow('w$, 'rowtime, 900000.millis)),
           term("select",
             "COUNT(*) AS EXPR$0, " +
               "weightedAvg(c, a) AS wAvg, " +
@@ -113,23 +115,23 @@ class WindowAggregateTest extends TableTestBase {
   }
 
   @Test
+  @Ignore // TODO enable once CALCITE-1761 is fixed
   def testHoppingFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
     val sql =
       "SELECT COUNT(*), weightedAvg(c, a) AS wAvg, " +
-        "  HOP_START(proctime(), INTERVAL '15' MINUTE, INTERVAL '1' HOUR), " +
-        "  HOP_END(proctime(), INTERVAL '15' MINUTE, INTERVAL '1' HOUR) " +
+        "  HOP_START(proctime, INTERVAL '15' MINUTE, INTERVAL '1' HOUR), " +
+        "  HOP_END(proctime, INTERVAL '15' MINUTE, INTERVAL '1' HOUR) " +
         "FROM MyTable " +
-        "GROUP BY HOP(proctime(), INTERVAL '15' MINUTE, INTERVAL '1' HOUR)"
+        "GROUP BY HOP(proctime, INTERVAL '15' MINUTE, INTERVAL '1' HOUR)"
     val expected =
       unaryNode(
         "DataStreamCalc",
         unaryNode(
           "DataStreamAggregate",
           streamTableNode(0),
-          term("window", ProcessingTimeSlidingGroupWindow('w$,
-            3600000.millis, 900000.millis)),
+          term("window", SlidingGroupWindow('w$, 'rowtime, 3600000.millis, 900000.millis)),
           term("select",
             "COUNT(*) AS EXPR$0, " +
               "weightedAvg(c, a) AS wAvg, " +
@@ -142,23 +144,24 @@ class WindowAggregateTest extends TableTestBase {
   }
 
   @Test
+  @Ignore // TODO enable once CALCITE-1761 is fixed
   def testSessionFunction() = {
     streamUtil.tEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge)
 
     val sql =
       "SELECT " +
         "  COUNT(*), weightedAvg(c, a) AS wAvg, " +
-        "  SESSION_START(proctime(), INTERVAL '15' MINUTE), " +
-        "  SESSION_END(proctime(), INTERVAL '15' MINUTE) " +
+        "  SESSION_START(proctime, INTERVAL '15' MINUTE), " +
+        "  SESSION_END(proctime, INTERVAL '15' MINUTE) " +
         "FROM MyTable " +
-        "GROUP BY SESSION(proctime(), INTERVAL '15' MINUTE)"
+        "GROUP BY SESSION(proctime, INTERVAL '15' MINUTE)"
     val expected =
       unaryNode(
         "DataStreamCalc",
         unaryNode(
           "DataStreamAggregate",
           streamTableNode(0),
-          term("window", ProcessingTimeSessionGroupWindow('w$, 900000.millis)),
+          term("window", SessionGroupWindow('w$, 'rowtime, 900000.millis)),
           term("select",
             "COUNT(*) AS EXPR$0, " +
               "weightedAvg(c, a) AS wAvg, " +
@@ -175,7 +178,7 @@ class WindowAggregateTest extends TableTestBase {
     val sqlQuery =
       "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " +
         "FROM MyTable " +
-        "GROUP BY TUMBLE(proctime(), INTERVAL '2' HOUR, TIME '10:00:00')"
+        "GROUP BY TUMBLE(proctime, INTERVAL '2' HOUR, TIME '10:00:00')"
 
     streamUtil.verifySql(sqlQuery, "n/a")
   }
@@ -185,7 +188,7 @@ class WindowAggregateTest extends TableTestBase {
     val sqlQuery =
       "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " +
         "FROM MyTable " +
-        "GROUP BY HOP(proctime(), INTERVAL '1' HOUR, INTERVAL '2' HOUR, TIME '10:00:00')"
+        "GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL '2' HOUR, TIME '10:00:00')"
 
     streamUtil.verifySql(sqlQuery, "n/a")
   }
@@ -195,21 +198,21 @@ class WindowAggregateTest extends TableTestBase {
     val sqlQuery =
       "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " +
         "FROM MyTable " +
-        "GROUP BY SESSION(proctime(), INTERVAL '2' HOUR, TIME '10:00:00')"
+        "GROUP BY SESSION(proctime, INTERVAL '2' HOUR, TIME '10:00:00')"
 
     streamUtil.verifySql(sqlQuery, "n/a")
   }
 
   @Test(expected = classOf[TableException])
   def testVariableWindowSize() = {
-    val sql = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime(), c * INTERVAL '1' MINUTE)"
+    val sql = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime, c * INTERVAL '1' MINUTE)"
     streamUtil.verifySql(sql, "n/a")
   }
 
   @Test(expected = classOf[TableException])
   def testMultiWindow() = {
     val sql = "SELECT COUNT(*) FROM MyTable GROUP BY " +
-      "FLOOR(rowtime() TO HOUR), FLOOR(rowtime() TO MINUTE)"
+      "FLOOR(rowtime TO HOUR), FLOOR(rowtime TO MINUTE)"
     val expected = ""
     streamUtil.verifySql(sql, expected)
   }
@@ -237,8 +240,8 @@ class WindowAggregateTest extends TableTestBase {
   def testUnboundPartitionedProcessingWindowWithRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (PARTITION BY c ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (PARTITION BY c ORDER BY proctime RANGE UNBOUNDED preceding) as cnt2 " +
       "from MyTable"
 
     val expected =
@@ -249,12 +252,12 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
+          term("select", "a", "c", "proctime", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
         ),
         term("select", "c", "w0$o0 AS cnt1", "CASE(>(w0$o0, 0)", "CAST(w0$o1), null) AS cnt2")
       )
@@ -265,7 +268,7 @@ class WindowAggregateTest extends TableTestBase {
   def testUnboundPartitionedProcessingWindowWithRow() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY ProcTime() ROWS BETWEEN UNBOUNDED preceding AND " +
+      "count(a) OVER (PARTITION BY c ORDER BY proctime ROWS BETWEEN UNBOUNDED preceding AND " +
       "CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -274,15 +277,11 @@ class WindowAggregateTest extends TableTestBase {
         "DataStreamCalc",
         unaryNode(
           "DataStreamOverAggregate",
-          unaryNode(
-            "DataStreamCalc",
-            streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
-          ),
+          streamTableNode(0),
           term("partitionBy", "c"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "b", "c", "proctime", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -293,8 +292,8 @@ class WindowAggregateTest extends TableTestBase {
   def testUnboundNonPartitionedProcessingWindowWithRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (ORDER BY ProcTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (ORDER BY proctime RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (ORDER BY proctime RANGE UNBOUNDED preceding) as cnt2 " +
       "from MyTable"
 
     val expected =
@@ -305,11 +304,11 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
+          term("select", "a", "c", "proctime", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
         ),
         term("select", "c", "w0$o0 AS cnt1", "CASE(>(w0$o0, 0)", "CAST(w0$o1), null) AS cnt2")
       )
@@ -320,7 +319,7 @@ class WindowAggregateTest extends TableTestBase {
   def testUnboundNonPartitionedProcessingWindowWithRow() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (ORDER BY ProcTime() ROWS BETWEEN UNBOUNDED preceding AND " +
+      "count(a) OVER (ORDER BY proctime ROWS BETWEEN UNBOUNDED preceding AND " +
       "CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -329,14 +328,10 @@ class WindowAggregateTest extends TableTestBase {
         "DataStreamCalc",
         unaryNode(
           "DataStreamOverAggregate",
-          unaryNode(
-            "DataStreamCalc",
-            streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
-          ),
-          term("orderBy", "PROCTIME"),
+          streamTableNode(0),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "b", "c", "proctime", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -347,8 +342,8 @@ class WindowAggregateTest extends TableTestBase {
   def testUnboundNonPartitionedEventTimeWindowWithRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (ORDER BY rowtime RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (ORDER BY rowtime RANGE UNBOUNDED preceding) as cnt2 " +
       "from MyTable"
 
     val expected =
@@ -359,11 +354,11 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
         ),
         term("select", "c", "w0$o0 AS cnt1", "CASE(>(w0$o0, 0)", "CAST(w0$o1), null) AS cnt2")
       )
@@ -374,8 +369,8 @@ class WindowAggregateTest extends TableTestBase {
   def testUnboundPartitionedEventTimeWindowWithRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt1, " +
-      "sum(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "count(a) OVER (PARTITION BY c ORDER BY rowtime RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (PARTITION BY c ORDER BY rowtime RANGE UNBOUNDED preceding) as cnt2 " +
       "from MyTable"
 
     val expected =
@@ -386,12 +381,12 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
         ),
         term("select", "c", "w0$o0 AS cnt1", "CASE(>(w0$o0, 0)", "CAST(w0$o1), null) AS cnt2")
       )
@@ -402,7 +397,7 @@ class WindowAggregateTest extends TableTestBase {
   def testBoundPartitionedRowTimeWindowWithRow() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 5 preceding AND " +
+      "count(a) OVER (PARTITION BY c ORDER BY rowtime ROWS BETWEEN 5 preceding AND " +
       "CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -414,12 +409,12 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("rows", "BETWEEN 5 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -430,7 +425,7 @@ class WindowAggregateTest extends TableTestBase {
   def testBoundNonPartitionedRowTimeWindowWithRow() = {
     val sql = "SELECT " +
         "c, " +
-        "count(a) OVER (ORDER BY RowTime() ROWS BETWEEN 5 preceding AND " +
+        "count(a) OVER (ORDER BY rowtime ROWS BETWEEN 5 preceding AND " +
         "CURRENT ROW) as cnt1 " +
         "from MyTable"
 
@@ -442,11 +437,11 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("rows", "BETWEEN 5 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -457,7 +452,7 @@ class WindowAggregateTest extends TableTestBase {
   def testBoundPartitionedRowTimeWindowWithRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY RowTime() " +
+      "count(a) OVER (PARTITION BY c ORDER BY rowtime " +
       "RANGE BETWEEN INTERVAL '1' SECOND  preceding AND CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -469,12 +464,12 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN 1000 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -485,7 +480,7 @@ class WindowAggregateTest extends TableTestBase {
   def testBoundNonPartitionedRowTimeWindowWithRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (ORDER BY RowTime() " +
+      "count(a) OVER (ORDER BY rowtime " +
       "RANGE BETWEEN INTERVAL '1' SECOND  preceding AND CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -497,11 +492,11 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "ROWTIME() AS $2")
+            term("select", "a", "c", "rowtime")
           ),
-          term("orderBy", "ROWTIME"),
+          term("orderBy", "rowtime"),
           term("range", "BETWEEN 1000 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "rowtime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -512,7 +507,7 @@ class WindowAggregateTest extends TableTestBase {
   def testBoundNonPartitionedProcTimeWindowWithRowRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
+      "count(a) OVER (ORDER BY proctime ROWS BETWEEN 2 preceding AND " +
       "CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -524,11 +519,11 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "proctime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )
@@ -539,7 +534,7 @@ class WindowAggregateTest extends TableTestBase {
   def testBoundPartitionedProcTimeWindowWithRowRange() = {
     val sql = "SELECT " +
       "c, " +
-      "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
+      "count(a) OVER (PARTITION BY c ORDER BY proctime ROWS BETWEEN 2 preceding AND " +
       "CURRENT ROW) as cnt1 " +
       "from MyTable"
 
@@ -551,12 +546,12 @@ class WindowAggregateTest extends TableTestBase {
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
-            term("select", "a", "c", "PROCTIME() AS $2")
+            term("select", "a", "c", "proctime")
           ),
           term("partitionBy", "c"),
-          term("orderBy", "PROCTIME"),
+          term("orderBy", "proctime"),
           term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
-          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+          term("select", "a", "c", "proctime", "COUNT(a) AS w0$o0")
         ),
         term("select", "c", "w0$o0 AS $1")
       )

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
index 3651749..4a6a616 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
@@ -57,13 +57,13 @@ class AggregationsITCase extends StreamingMultipleProgramsTestBase {
     StreamITCase.testResults = mutable.MutableList()
 
     val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime)
 
     val countFun = new CountAggFunction
     val weightAvgFun = new WeightedAvg
 
     val windowedTable = table
-      .window(Slide over 2.rows every 1.rows as 'w)
+      .window(Slide over 2.rows every 1.rows on 'proctime as 'w)
       .groupBy('w, 'string)
       .select('string, countFun('int), 'int.avg,
               weightAvgFun('long, 'int), weightAvgFun('int, 'int))
@@ -102,7 +102,7 @@ class AggregationsITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(sessionWindowTestdata)
       .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset(10L))
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime)
 
     val windowedTable = table
       .window(Session withGap 5.milli on 'rowtime as 'w)
@@ -126,12 +126,12 @@ class AggregationsITCase extends StreamingMultipleProgramsTestBase {
     StreamITCase.testResults = mutable.MutableList()
 
     val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime)
     val countFun = new CountAggFunction
     val weightAvgFun = new WeightedAvg
 
     val windowedTable = table
-      .window(Tumble over 2.rows as 'w)
+      .window(Tumble over 2.rows on 'proctime as 'w)
       .groupBy('w)
       .select(countFun('string), 'int.avg,
               weightAvgFun('long, 'int), weightAvgFun('int, 'int))
@@ -154,7 +154,7 @@ class AggregationsITCase extends StreamingMultipleProgramsTestBase {
     val stream = env
       .fromCollection(data)
       .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset(0L))
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime)
     val countFun = new CountAggFunction
     val weightAvgFun = new WeightedAvg
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
index 5969e91..1114cf0 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
@@ -111,22 +111,6 @@ class CalcITCase extends StreamingMultipleProgramsTestBase {
   }
 
   @Test(expected = classOf[TableException])
-  def testAsWithToFewFields(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("no")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test(expected = classOf[TableException])
   def testAsWithToManyFields(): Unit = {
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment


[08/15] flink git commit: [FLINK-6216] [table] Add non-windowed GroupBy aggregation for streams.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowAggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowAggregationsITCase.scala
new file mode 100644
index 0000000..2c027a9
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowAggregationsITCase.scala
@@ -0,0 +1,197 @@
+/*
+ * 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.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+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.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.java.utils.UserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.table.GroupWindowAggregationsITCase.TimestampAndWatermarkWithOffset
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase
+import org.apache.flink.table.functions.aggfunctions.CountAggFunction
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+/**
+  * We only test some aggregations until better testing of constructed DataStream
+  * programs is possible.
+  */
+class GroupWindowAggregationsITCase extends StreamingMultipleProgramsTestBase {
+
+  val data = List(
+    (1L, 1, "Hi"),
+    (2L, 2, "Hello"),
+    (4L, 2, "Hello"),
+    (8L, 3, "Hello world"),
+    (16L, 3, "Hello world"))
+
+  @Test
+  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime)
+
+    val countFun = new CountAggFunction
+    val weightAvgFun = new WeightedAvg
+
+    val windowedTable = table
+      .window(Slide over 2.rows every 1.rows on 'proctime as 'w)
+      .groupBy('w, 'string)
+      .select('string, countFun('int), 'int.avg,
+              weightAvgFun('long, 'int), weightAvgFun('int, 'int))
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("Hello world,1,3,8,3", "Hello world,2,3,12,3", "Hello,1,2,2,2",
+                       "Hello,2,2,3,2", "Hi,1,1,1,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testEventTimeSessionGroupWindowOverTime(): Unit = {
+    //To verify the "merge" functionality, we create this test with the following characteristics:
+    // 1. set the Parallelism to 1, and have the test data out of order
+    // 2. create a waterMark with 10ms offset to delay the window emission by 10ms
+    val sessionWindowTestdata = List(
+      (1L, 1, "Hello"),
+      (2L, 2, "Hello"),
+      (8L, 8, "Hello"),
+      (9L, 9, "Hello World"),
+      (4L, 4, "Hello"),
+      (16L, 16, "Hello"))
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val countFun = new CountAggFunction
+    val weightAvgFun = new WeightedAvgWithMerge
+
+    val stream = env
+      .fromCollection(sessionWindowTestdata)
+      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset(10L))
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime)
+
+    val windowedTable = table
+      .window(Session withGap 5.milli on 'rowtime as 'w)
+      .groupBy('w, 'string)
+      .select('string, countFun('int), 'int.avg,
+              weightAvgFun('long, 'int), weightAvgFun('int, 'int))
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("Hello World,1,9,9,9", "Hello,1,16,16,16", "Hello,4,3,5,5")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime)
+    val countFun = new CountAggFunction
+    val weightAvgFun = new WeightedAvg
+
+    val windowedTable = table
+      .window(Tumble over 2.rows on 'proctime as 'w)
+      .groupBy('w)
+      .select(countFun('string), 'int.avg,
+              weightAvgFun('long, 'int), weightAvgFun('int, 'int))
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("2,1,1,1", "2,2,6,2")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testEventTimeTumblingWindow(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val stream = env
+      .fromCollection(data)
+      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset(0L))
+    val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime)
+    val countFun = new CountAggFunction
+    val weightAvgFun = new WeightedAvg
+
+    val windowedTable = table
+      .window(Tumble over 5.milli on 'rowtime as 'w)
+      .groupBy('w, 'string)
+      .select('string, countFun('string), 'int.avg, weightAvgFun('long, 'int),
+              weightAvgFun('int, 'int), 'int.min, 'int.max, 'int.sum, 'w.start, 'w.end)
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "Hello world,1,3,8,3,3,3,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01",
+      "Hello world,1,3,16,3,3,3,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02",
+      "Hello,2,2,3,2,2,2,4,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005",
+      "Hi,1,1,1,1,1,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+}
+
+object GroupWindowAggregationsITCase {
+  class TimestampAndWatermarkWithOffset(
+    offset: Long) extends AssignerWithPunctuatedWatermarks[(Long, Int, String)] {
+
+    override def checkAndGetNextWatermark(
+        lastElement: (Long, Int, String),
+        extractedTimestamp: Long)
+      : Watermark = {
+      new Watermark(extractedTimestamp - offset)
+    }
+
+    override def extractTimestamp(
+        element: (Long, Int, String),
+        previousElementTimestamp: Long): Long = {
+      element._1
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/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 de6cbfa..0573ff3 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,11 +185,11 @@ class GroupWindowTest extends TableTestBase {
       .select('string.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         unaryNode(
-          "DataStreamAggregate",
+          "DataStreamGroupWindowAggregate",
           unaryNode(
             "DataStreamCalc",
             streamTableNode(0),
@@ -229,7 +229,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -259,7 +259,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -268,10 +268,7 @@ class GroupWindowTest extends TableTestBase {
       term("groupBy", "string"),
       term(
         "window",
-        TumblingGroupWindow(
-          WindowReference("w"),
-          'proctime,
-          2.rows)),
+        TumblingGroupWindow(WindowReference("w"), 'proctime, 2.rows)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -289,7 +286,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       streamTableNode(0),
       term("groupBy", "string"),
       term(
@@ -317,7 +314,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, weightedAvg('long, 'int))
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       streamTableNode(0),
       term("groupBy", "string"),
       term(
@@ -343,7 +340,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -374,7 +371,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -397,6 +394,32 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testEventTimeSlidingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
+
+    val windowedTable = table
+      .window(Slide over 8.milli every 10.milli on 'rowtime as 'w)
+      .groupBy('w, 'string)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamGroupWindowAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int", "rowtime")
+      ),
+      term("groupBy", "string"),
+      term("window", SlidingGroupWindow(WindowReference("w"), 'rowtime, 8.milli, 10.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  @Ignore // see comments in DataStreamGroupWindowAggregate
+  def testEventTimeSlidingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
     val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
@@ -405,7 +428,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       streamTableNode(0),
       term("groupBy", "string"),
       term(
@@ -434,12 +457,10 @@ class GroupWindowTest extends TableTestBase {
       .select('string, weightedAvg('long, 'int))
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       streamTableNode(0),
       term("groupBy", "string"),
-      term(
-        "window",
-        SlidingGroupWindow(WindowReference("w"), 'rowtime, 8.milli, 10.milli)),
+      term("window", SlidingGroupWindow(WindowReference("w"), 'rowtime, 8.milli, 10.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -457,15 +478,10 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       streamTableNode(0),
       term("groupBy", "string"),
-      term(
-        "window",
-        SessionGroupWindow(
-          WindowReference("w"),
-          'long,
-          7.milli)),
+      term("window", SessionGroupWindow(WindowReference("w"), 'long, 7.milli)),
       term("select", "string", "COUNT(int) AS TMP_0")
     )
 
@@ -485,12 +501,10 @@ class GroupWindowTest extends TableTestBase {
       .select('string, weightedAvg('long, 'int))
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       streamTableNode(0),
       term("groupBy", "string"),
-      term(
-        "window",
-        SessionGroupWindow(WindowReference("w"), 'rowtime, 7.milli)),
+      term("window", SessionGroupWindow(WindowReference("w"), 'rowtime, 7.milli)),
       term("select", "string", "WeightedAvgWithMerge(long, int) AS TMP_0")
     )
 
@@ -508,7 +522,7 @@ class GroupWindowTest extends TableTestBase {
       .select('string, 'int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -538,7 +552,7 @@ class GroupWindowTest extends TableTestBase {
       .select('int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -559,6 +573,31 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllEventTimeTumblingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
+
+    val windowedTable = table
+      .window(Tumble over 5.milli on 'rowtime as 'w)
+      .groupBy('w)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamGroupWindowAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int", "rowtime")
+      ),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'rowtime, 5.milli)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  @Ignore // see comments in DataStreamGroupWindowAggregate
+  def testAllEventTimeTumblingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
     val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
@@ -567,7 +606,7 @@ class GroupWindowTest extends TableTestBase {
       .select('int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -596,7 +635,7 @@ class GroupWindowTest extends TableTestBase {
       .select('int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -626,7 +665,7 @@ class GroupWindowTest extends TableTestBase {
       .select('int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -648,6 +687,31 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testAllEventTimeSlidingGroupWindowOverTime(): Unit = {
     val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
+
+    val windowedTable = table
+      .window(Slide over 8.milli every 10.milli on 'rowtime as 'w)
+      .groupBy('w)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamGroupWindowAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int", "rowtime")
+      ),
+      term("window", SlidingGroupWindow(WindowReference("w"), 'rowtime, 8.milli, 10.milli)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+//  @Ignore // see comments in DataStreamGroupWindowAggregate
+  def testAllEventTimeSlidingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
     val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
 
     val windowedTable = table
@@ -656,19 +720,13 @@ class GroupWindowTest extends TableTestBase {
       .select('int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
         term("select", "int", "long")
       ),
-      term(
-        "window",
-        SlidingGroupWindow(
-          WindowReference("w"),
-          'long,
-          8.milli,
-          10.milli)),
+      term("window", SlidingGroupWindow(WindowReference("w"), 'long, 8.milli, 10.milli)),
       term("select", "COUNT(int) AS TMP_0")
     )
 
@@ -686,7 +744,7 @@ class GroupWindowTest extends TableTestBase {
       .select('int.count)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
+      "DataStreamGroupWindowAggregate",
       unaryNode(
         "DataStreamCalc",
         streamTableNode(0),
@@ -707,22 +765,22 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testTumbleWindowStartEnd(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
 
     val windowedTable = table
-      .window(Tumble over 5.milli on 'long as 'w)
+      .window(Tumble over 5.milli on 'rowtime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
-      streamTableNode(0),
+      "DataStreamGroupWindowAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int", "rowtime")
+      ),
       term("groupBy", "string"),
-      term("window",
-        TumblingGroupWindow(
-          WindowReference("w"),
-          'long,
-          5.milli)),
+      term("window", TumblingGroupWindow(WindowReference("w"), 'rowtime, 5.milli)),
       term("select",
         "string",
         "COUNT(int) AS TMP_0",
@@ -736,23 +794,22 @@ class GroupWindowTest extends TableTestBase {
   @Test
   def testSlideWindowStartEnd(): Unit = {
     val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long.rowtime, 'int, 'string)
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string, 'rowtime.rowtime)
 
     val windowedTable = table
-      .window(Slide over 10.milli every 5.milli on 'long as 'w)
+      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
       .groupBy('w, 'string)
       .select('string, 'int.count, 'w.start, 'w.end)
 
     val expected = unaryNode(
-      "DataStreamAggregate",
-      streamTableNode(0),
+      "DataStreamGroupWindowAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int", "rowtime")
+      ),
       term("groupBy", "string"),
-      term("window",
-        SlidingGroupWindow(
-          WindowReference("w"),
-          'long,
-          10.milli,
-          5.milli)),
+      term("window", SlidingGroupWindow(WindowReference("w"), 'rowtime, 10.milli, 5.milli)),
       term("select",
         "string",
         "COUNT(int) AS TMP_0",
@@ -776,14 +833,10 @@ class GroupWindowTest extends TableTestBase {
     val expected = unaryNode(
       "DataStreamCalc",
       unaryNode(
-        "DataStreamAggregate",
+        "DataStreamGroupWindowAggregate",
         streamTableNode(0),
         term("groupBy", "string"),
-        term("window",
-          SessionGroupWindow(
-            WindowReference("w"),
-            'long,
-            3.milli)),
+        term("window", SessionGroupWindow(WindowReference("w"), 'long, 3.milli)),
         term("select",
           "string",
           "COUNT(int) AS TMP_1",
@@ -810,14 +863,10 @@ class GroupWindowTest extends TableTestBase {
     val expected = unaryNode(
       "DataStreamCalc",
       unaryNode(
-        "DataStreamAggregate",
+        "DataStreamGroupWindowAggregate",
         streamTableNode(0),
         term("groupBy", "string"),
-        term("window",
-          TumblingGroupWindow(
-            WindowReference("w"),
-            'long,
-            5.millis)),
+        term("window", TumblingGroupWindow(WindowReference("w"), 'long, 5.millis)),
         term("select",
           "string",
           "SUM(int) AS TMP_0",

http://git-wip-us.apache.org/repos/asf/flink/blob/8f78824b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
index b6a6660..c72249a 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
@@ -28,13 +28,6 @@ import org.junit.Test
 class UnsupportedOpsTest extends StreamingMultipleProgramsTestBase {
 
   @Test(expected = classOf[ValidationException])
-  def testSelectWithAggregation(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1.min)
-  }
-
-  @Test(expected = classOf[ValidationException])
   def testDistinct(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)


[14/15] flink git commit: [FLINK-6093] [table] Add stream TableSinks and DataStream conversion with support for retraction.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
index 809afd2..7214394 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
@@ -25,7 +25,6 @@ import org.apache.flink.types.Row
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
 
 /**
   * A simple [[TableSink]] to emit data as CSV files.
@@ -40,7 +39,7 @@ class CsvTableSink(
     fieldDelim: Option[String],
     numFiles: Option[Int],
     writeMode: Option[WriteMode])
-  extends TableSinkBase[Row] with BatchTableSink[Row] with StreamTableSink[Row] {
+  extends TableSinkBase[Row] with BatchTableSink[Row] with AppendStreamTableSink[Row] {
 
   /**
     * A simple [[TableSink]] to emit data as CSV files.
@@ -134,100 +133,3 @@ class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] {
     builder.mkString
   }
 }
-
-/**
-  * A simple [[TableSink]] to emit data as CSV files.
-  *
-  * @param path The output path to write the Table to.
-  * @param fieldDelim The field delimiter
-  * @param numFiles The number of files to write to
-  * @param writeMode The write mode to specify whether existing files are overwritten or not.
-  */
-class CsvRetractTableSink(
-    path: String,
-    fieldDelim: Option[String],
-    numFiles: Option[Int],
-    writeMode: Option[WriteMode])
-  extends TableSinkBase[Row] with StreamRetractSink[Row] {
-
-  override def needsUpdatesAsRetraction: Boolean = true
-
-  /**
-    * A simple [[TableSink]] to emit data as CSV files.
-    *
-    * @param path The output path to write the Table to.
-    * @param fieldDelim The field delimiter, ',' by default.
-    */
-  def this(path: String, fieldDelim: String = ",") {
-    this(path, Some(fieldDelim), None, None)
-  }
-
-  /**
-    * A simple [[TableSink]] to emit data as CSV files.
-    *
-    * @param path The output path to write the Table to.
-    * @param fieldDelim The field delimiter.
-    * @param numFiles The number of files to write to.
-    * @param writeMode The write mode to specify whether existing files are overwritten or not.
-    */
-  def this(path: String, fieldDelim: String, numFiles: Int, writeMode: WriteMode) {
-    this(path, Some(fieldDelim), Some(numFiles), Some(writeMode))
-  }
-
-
-  override def emitDataStreamWithChange(dataStream: DataStream[JTuple2[Boolean,Row]]): Unit = {
-    val csvRows = dataStream
-      .map(new CsvRetractFormatter(fieldDelim.getOrElse(",")))
-      .returns(TypeInformation.of(classOf[String]))
-
-
-    if (numFiles.isDefined) {
-      csvRows.setParallelism(numFiles.get)
-    }
-
-    val sink = writeMode match {
-      case None => csvRows.writeAsText(path)
-      case Some(wm) => csvRows.writeAsText(path, wm)
-    }
-
-    if (numFiles.isDefined) {
-      sink.setParallelism(numFiles.get)
-    }
-  }
-
-  override protected def copy: TableSinkBase[Row] = {
-    new CsvRetractTableSink(path, fieldDelim, numFiles, writeMode)
-  }
-
-  override def getOutputType: TypeInformation[Row] = {
-    new RowTypeInfo(getFieldTypes: _*)
-  }
-}
-
-/**
-  * Formats a [[Tuple2]] with change information into a [[String]] with fields separated by the
-  * field delimiter.
-  *
-  * @param fieldDelim The field delimiter.
-  */
-class CsvRetractFormatter(fieldDelim: String) extends MapFunction[JTuple2[Boolean,Row], String] {
-  override def map(rowT: JTuple2[Boolean,Row]): String = {
-
-    val row: Row = rowT.f1
-
-    val builder = new StringBuilder
-
-    builder.append(rowT.f0.toString)
-
-    // write following values
-    for (i <- 0 until row.getArity) {
-      builder.append(fieldDelim)
-      val v = row.getField(i)
-      if (v != null) {
-        builder.append(v.toString)
-      }
-    }
-    builder.mkString
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
new file mode 100644
index 0000000..3ab997e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.table.sinks
+
+import java.lang.{Boolean => JBool}
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.table.api.Types
+
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.Table
+
+/**
+  * Defines an external [[TableSink]] to emit a streaming [[Table]] with insert, update, and delete
+  * changes.
+  *
+  * The table will be converted into a stream of accumulate and retraction messages which are
+  * encoded as [[JTuple2]].
+  * The first field is a [[JBool]] flag to indicate the message type.
+  * The second field holds the record of the requested type [[T]].
+  *
+  * A message with true [[JBool]] flag is an accumulate (or add) message.
+  * A message with false flag is a retract message.
+  *
+  * @tparam T Type of records that this [[TableSink]] expects and supports.
+  */
+trait RetractStreamTableSink[T] extends TableSink[JTuple2[JBool, T]] {
+
+  /** Returns the requested record type */
+  def getRecordType: TypeInformation[T]
+
+  /** Emits the DataStream. */
+  def emitDataStream(dataStream: DataStream[JTuple2[JBool, T]]): Unit
+
+  override def getOutputType = new TupleTypeInfo(Types.BOOLEAN, getRecordType)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala
deleted file mode 100644
index 7f7c944..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.table.sinks
-
-import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
-import org.apache.flink.streaming.api.datastream.DataStream
-
-trait StreamRetractSink[T] extends TableSink[T]{
-
-  /**
-    * Whether the [[StreamTableSink]] requires that update and delete changes are sent with
-    * retraction messages.
-    */
-  def needsUpdatesAsRetraction: Boolean = false
-
-  /** Emits the DataStream with change infomation. */
-  def emitDataStreamWithChange(dataStream: DataStream[JTuple2[Boolean,T]]): Unit
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala
deleted file mode 100644
index 360252e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.table.sinks
-
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.Table
-
-/** Defines an external [[TableSink]] to emit a batch [[Table]].
-  *
-  * @tparam T Type of [[DataStream]] that this [[TableSink]] expects and supports.
-  */
-trait StreamTableSink[T] extends TableSink[T] {
-
-  /** Emits the DataStream. */
-  def emitDataStream(dataStream: DataStream[T]): Unit
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
new file mode 100644
index 0000000..2ae3406
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
@@ -0,0 +1,79 @@
+/*
+ * 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.table.sinks
+
+import java.lang.{Boolean => JBool}
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{Table, Types}
+
+/**
+  * Defines an external [[TableSink]] to emit a streaming [[Table]] with insert, update, and delete
+  * changes. The [[Table]] must be have unique key fields (atomic or composite) or be append-only.
+  *
+  * If the [[Table]] does not have a unique key and is not append-only, a
+  * [[org.apache.flink.table.api.TableException]] will be thrown.
+  *
+  * The unique key of the table is configured by the [[UpsertStreamTableSink#setKeyFields()]]
+  * method.
+  *
+  * The [[Table]] will be converted into a stream of upsert and delete messages which are encoded as
+  * [[JTuple2]]. The first field is a [[JBool]] flag to indicate the message type. The second field
+  * holds the record of the requested type [[T]].
+  *
+  * A message with true [[JBool]] field is an upsert message for the configured key.
+  * A message with false flag is a delete message for the configured key.
+  *
+  * If the table is append-only, all messages will have a true flag and must be interpreted
+  * as insertions.
+  *
+  * @tparam T Type of records that this [[TableSink]] expects and supports.
+  */
+trait UpsertStreamTableSink[T] extends TableSink[JTuple2[JBool, T]] {
+
+  /**
+    * Configures the unique key fields of the [[Table]] to write.
+    * The method is called after [[TableSink.configure()]].
+    *
+    * The keys array might be empty, if the table consists of a single (updated) record.
+    * If the table does not have a key and is append-only, the keys attribute is null.
+    *
+    * @param keys the field names of the table's keys, an empty array if the table has a single
+    *             row, and null if the table is append-only and has no key.
+    */
+  def setKeyFields(keys: Array[String]): Unit
+
+  /**
+    * Specifies whether the [[Table]] to write is append-only or not.
+    *
+    * @param isAppendOnly true if the table is append-only, false otherwise.
+    */
+  def setIsAppendOnly(isAppendOnly: Boolean): Unit
+
+  /** Returns the requested record type */
+  def getRecordType: TypeInformation[T]
+
+  /** Emits the DataStream. */
+  def emitDataStream(dataStream: DataStream[JTuple2[JBool, T]]): Unit
+
+  override def getOutputType = new TupleTypeInfo(Types.BOOLEAN, getRecordType)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
index 675e5d9..ba3b591 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
@@ -21,7 +21,6 @@ package org.apache.flink.table
 import org.apache.flink.api.scala._
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
 import org.apache.flink.table.api.TableException

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
index d490763..40f4c7d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
@@ -27,10 +27,8 @@ import org.junit.Test
 import org.apache.flink.table.api.scala._
 import org.apache.flink.api.scala._
 import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.table.utils.TableFunc0
 
-import scala.collection.mutable
 
 /**
   * tests for retraction
@@ -55,51 +53,47 @@ class RetractionITCase extends StreamingWithStateTestBase {
   def testWordCount(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    env.setParallelism(1)
+    StreamITCase.clear
     env.setStateBackend(getStateBackend)
 
     val stream = env.fromCollection(data)
     val table = stream.toTable(tEnv, 'word, 'num)
     val resultTable = table
       .groupBy('word)
-      .select('word as 'word, 'num.sum as 'count)
+      .select('num.sum as 'count)
       .groupBy('count)
-      .select('count, 'word.count as 'frequency)
+      .select('count, 'count.count as 'frequency)
 
-    val results = resultTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = resultTable.toRetractStream[Row]
+    results.addSink(new StreamITCase.RetractingSink)
     env.execute()
 
-    val expected = Seq("1,1", "1,2", "1,1", "2,1", "1,2", "1,1", "2,2", "2,1", "3,1", "3,0",
-      "4,1", "4,0", "5,1", "5,0", "6,1", "1,2")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = Seq("1,2", "2,1", "6,1")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   // keyed groupby + non-keyed groupby
   @Test
   def testGroupByAndNonKeyedGroupBy(): Unit = {
-
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    env.setParallelism(1)
+    StreamITCase.clear
     env.setStateBackend(getStateBackend)
 
     val stream = env.fromCollection(data)
     val table = stream.toTable(tEnv, 'word, 'num)
     val resultTable = table
       .groupBy('word)
-      .select('word as 'word, 'num.sum as 'count)
-      .select('count.sum)
+      .select('word as 'word, 'num.sum as 'cnt)
+      .select('cnt.sum)
+
+    val results = resultTable.toRetractStream[Row]
 
-    val results = resultTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    results.addSink(new StreamITCase.RetractingSink).setParallelism(1)
     env.execute()
 
-    val expected = Seq("1", "2", "1", "3", "4", "3", "5", "3", "6", "3", "7", "3", "8", "3", "9",
-      "10")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = Seq("10")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   // non-keyed groupby + keyed groupby
@@ -108,8 +102,7 @@ class RetractionITCase extends StreamingWithStateTestBase {
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    env.setParallelism(1)
+    StreamITCase.clear
     env.setStateBackend(getStateBackend)
 
     val stream = env.fromCollection(data)
@@ -119,13 +112,12 @@ class RetractionITCase extends StreamingWithStateTestBase {
       .groupBy('count)
       .select('count, 'count.count)
 
-    val results = resultTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = resultTable.toRetractStream[Row]
+    results.addSink(new StreamITCase.RetractingSink)
     env.execute()
 
-    val expected = Seq("1,1", "1,0", "2,1", "2,0", "3,1", "3,0", "4,1", "4,0", "5,1", "5,0", "6," +
-      "1", "6,0", "7,1", "7,0", "8,1", "8,0", "9,1", "9,0", "10,1")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = Seq("10,1")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   // test unique process, if the current output message of unbounded groupby equals the
@@ -150,9 +142,9 @@ class RetractionITCase extends StreamingWithStateTestBase {
     )
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    env.setParallelism(1)
+    StreamITCase.clear
     env.setStateBackend(getStateBackend)
+    env.setParallelism(1)
 
     val stream = env.fromCollection(data)
     val table = stream.toTable(tEnv, 'pk, 'value)
@@ -162,12 +154,13 @@ class RetractionITCase extends StreamingWithStateTestBase {
       .groupBy('sum)
       .select('sum, 'pk.count as 'count)
 
-    val results = resultTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = resultTable.toRetractStream[Row]
+    results.addSink(new StreamITCase.RetractMessagesSink)
     env.execute()
 
-    val expected = Seq("1,1", "2,1", "3,1", "3,0", "6,1", "1,2", "1,3", "6,2", "6,1", "12,1","12," +
-      "0", "18,1", "8,1")
+    val expected = Seq(
+      "+1,1", "+2,1", "+3,1", "-3,1", "+6,1", "-1,1", "+1,2", "-1,2", "+1,3", "-6,1", "+6,2",
+      "-6,2", "+6,1", "+12,1", "-12,1", "+18,1", "+8,1")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -176,8 +169,7 @@ class RetractionITCase extends StreamingWithStateTestBase {
   def testCorrelate(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    env.setParallelism(1)
+    StreamITCase.clear
     env.setStateBackend(getStateBackend)
 
     val func0 = new TableFunc0
@@ -186,19 +178,17 @@ class RetractionITCase extends StreamingWithStateTestBase {
     val table = stream.toTable(tEnv, 'word, 'num)
     val resultTable = table
       .groupBy('word)
-      .select('word as 'word, 'num.sum as 'count)
+      .select('word as 'word, 'num.sum as 'cnt)
       .leftOuterJoin(func0('word))
-      .groupBy('count)
-      .select('count, 'word.count as 'frequency)
+      .groupBy('cnt)
+      .select('cnt, 'word.count as 'frequency)
 
-    val results = resultTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = resultTable.toRetractStream[Row]
+    results.addSink(new StreamITCase.RetractingSink)
     env.execute()
 
-    val expected = Seq(
-      "1,1", "1,2", "1,1", "2,1", "1,2", "1,1", "2,2", "2,1", "3,1", "3,0", "4,1", "4,0", "5,1",
-      "5,0", "6,1", "1,2")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = Seq("1,2", "2,1", "6,1")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
index ceae6c6..c446d64 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
@@ -23,7 +23,7 @@ import java.io.File
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala.stream.utils.StreamTestData
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sinks.{CsvTableSink, CsvRetractTableSink}
+import org.apache.flink.table.sinks.CsvTableSink
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.table.api.TableEnvironment
@@ -59,34 +59,5 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase {
 
     TestBaseUtils.compareResultsByLinesInMemory(expected, path)
   }
-
-  @Test
-  def testStreamTableSinkNeedRetraction(): Unit = {
-
-    val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
-    tmpFile.deleteOnExit()
-    val path = tmpFile.toURI.toString
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    env.setParallelism(4)
-
-    val input = StreamTestData.get3TupleDataStream(env)
-      .map(x => x).setParallelism(1) // increase DOP to 4
-
-    val results = input.toTable(tEnv, 'a, 'b, 'c)
-      .where('a < 5 || 'a > 17)
-      .select('c, 'b)
-      .groupBy('b)
-      .select('b, 'c.count)
-      .writeToSink(new CsvRetractTableSink(path))
-
-    env.execute()
-
-    val expected = Seq(
-      "true,1,1", "true,2,1", "false,2,1", "true,2,2", "true,3,1", "true,6,1", "false,6,1",
-      "true,6,2", "false,6,2", "true,6,3", "false,6,3", "true,6,4").mkString("\n")
-
-    TestBaseUtils.compareResultsByLinesInMemory(expected, path)
-  }
+  
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index abbcbdd..249d505 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -53,19 +53,19 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT b, COUNT(a) FROM MyTable GROUP BY b"
 
-    val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
     tEnv.registerTable("MyTable", t)
 
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
+    val result = tEnv.sql(sqlQuery).toRetractStream[Row]
+    result.addSink(new StreamITCase.RetractingSink).setParallelism(1)
     env.execute()
 
-    val expected = mutable.MutableList("1,1", "2,1", "2,2")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = List("1,1", "2,2", "3,3", "4,4", "5,5", "6,6")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   /** test selection **/
@@ -74,7 +74,7 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT a * 2, b - 1 FROM MyTable"
 
@@ -85,7 +85,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList("2,0", "4,1", "6,1")
+    val expected = List("2,0", "4,1", "6,1")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -95,7 +95,7 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT * FROM MyTable WHERE a = 3"
 
@@ -106,7 +106,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList("3,2,Hello world")
+    val expected = List("3,2,Hello world")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -116,7 +116,7 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT * FROM MyTable WHERE _1 = 3"
 
@@ -127,7 +127,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList("3,2,Hello world")
+    val expected = List("3,2,Hello world")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -136,7 +136,7 @@ class SqlITCase extends StreamingWithStateTestBase {
   def testUnion(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT * FROM T1 " +
       "UNION ALL " +
@@ -151,7 +151,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,1,Hi", "1,1,Hi",
       "2,2,Hello", "2,2,Hello",
       "3,2,Hello world", "3,2,Hello world")
@@ -163,7 +163,7 @@ class SqlITCase extends StreamingWithStateTestBase {
   def testUnionWithFilter(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT * FROM T1 WHERE a = 3 " +
       "UNION ALL " +
@@ -178,7 +178,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "2,2,Hello",
       "3,2,Hello world")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
@@ -189,7 +189,7 @@ class SqlITCase extends StreamingWithStateTestBase {
   def testUnionTableWithDataSet(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT c FROM T1 WHERE a = 3 " +
       "UNION ALL " +
@@ -204,7 +204,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList("Hello", "Hello world")
+    val expected = List("Hello", "Hello world")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -213,7 +213,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     // for sum aggregation ensure that every time the order of each element is consistent
     env.setParallelism(1)
@@ -232,7 +232,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello World,1,7", "Hello World,2,15", "Hello World,3,35",
       "Hello,1,1", "Hello,2,3", "Hello,3,6", "Hello,4,10", "Hello,5,15", "Hello,6,21")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
@@ -243,7 +243,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
@@ -259,7 +259,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello World,1", "Hello World,2", "Hello World,3",
       "Hello,1", "Hello,2", "Hello,3", "Hello,4", "Hello,5", "Hello,6")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
@@ -270,7 +270,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     // for sum aggregation ensure that every time the order of each element is consistent
     env.setParallelism(1)
@@ -289,7 +289,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello World,7,28", "Hello World,8,36", "Hello World,9,56",
       "Hello,1,1", "Hello,2,3", "Hello,3,6", "Hello,4,10", "Hello,5,15", "Hello,6,21")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
@@ -300,7 +300,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
@@ -314,7 +314,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList("1", "2", "3", "4", "5", "6", "7", "8", "9")
+    val expected = List("1", "2", "3", "4", "5", "6", "7", "8", "9")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -363,7 +363,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello,1,1,1", "Hello,1,2,2", "Hello,1,3,3",
       "Hello,2,3,4", "Hello,2,3,5", "Hello,2,3,6",
       "Hello,3,3,7", "Hello,4,3,9", "Hello,5,3,12",
@@ -420,7 +420,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello,1,1,1", "Hello,1,2,2", "Hello,1,3,3",
       "Hello,2,3,4", "Hello,2,3,5", "Hello,2,3,6",
       "Hello,3,3,7",
@@ -490,7 +490,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3",
       "Hello,2,6,9", "Hello,3,6,9", "Hello,2,6,9",
       "Hello,3,4,9",
@@ -562,7 +562,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3",
       "Hello,2,6,9", "Hello,3,6,9", "Hello,2,6,9",
       "Hello,3,4,9",
@@ -584,7 +584,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
@@ -608,7 +608,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setStateBackend(getStateBackend)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
@@ -661,7 +661,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,2,Hello,2,1,2,2,2",
       "1,3,Hello world,5,2,2,3,2",
       "1,1,Hi,6,3,2,3,1",
@@ -687,7 +687,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setStateBackend(getStateBackend)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val sqlQuery = "SELECT a, b, c, " +
       "SUM(b) over (" +
@@ -731,7 +731,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,2,Hello,2,1,2,2,2",
       "1,3,Hello world,5,2,2,3,2",
       "1,1,Hi,6,3,2,3,1",
@@ -757,7 +757,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setStateBackend(getStateBackend)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
@@ -793,7 +793,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "2,2,Hello,2,1,2,2,2",
       "3,5,Hello,7,2,3,5,2",
       "1,3,Hello,10,3,3,5,2",
@@ -812,7 +812,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setStateBackend(getStateBackend)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
@@ -849,7 +849,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "2,2,Hello,2,1,2,2,2",
       "3,5,Hello,7,2,3,5,2",
       "1,3,Hello,10,3,3,5,2",
@@ -869,7 +869,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setStateBackend(getStateBackend)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
@@ -907,7 +907,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "2,1,Hello,1,1,1,1,1",
       "1,1,Hello,7,4,1,3,1",
       "1,2,Hello,7,4,1,3,1",
@@ -932,7 +932,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setStateBackend(getStateBackend)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
@@ -975,7 +975,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,1,Hello,6,3,2,3,1",
       "1,2,Hello,6,3,2,3,1",
       "1,3,Hello world,6,3,2,3,1",
@@ -1000,7 +1000,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setParallelism(1)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t = StreamTestData.get5TupleDataStream(env)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
@@ -1017,7 +1017,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,0,0",
       "2,1,1",
       "2,3,1",
@@ -1043,7 +1043,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setParallelism(1)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t = StreamTestData.get5TupleDataStream(env)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
@@ -1060,7 +1060,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,0,0",
       "2,1,1",
       "2,3,1",
@@ -1087,7 +1087,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setParallelism(1)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t = StreamTestData.get5TupleDataStream(env)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
@@ -1104,7 +1104,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,0,0",
       "2,1,0",
       "2,3,0",
@@ -1130,7 +1130,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setStateBackend(getStateBackend)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     env.setParallelism(1)
-    StreamITCase.testResults = mutable.MutableList()
+    StreamITCase.clear
 
     val t = StreamTestData.get5TupleDataStream(env)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime)
@@ -1146,7 +1146,7 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = mutable.MutableList(
+    val expected = List(
       "1,0,0",
       "2,1,0",
       "2,3,0",

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
index 271e90b..910cbf2 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
@@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase}
 import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase.RetractingSink
 import org.apache.flink.types.Row
 import org.junit.Assert.assertEquals
 import org.junit.Test
@@ -38,29 +39,24 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
   def testNonKeyedGroupAggregate(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
-    env.setParallelism(1)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.clear
 
     val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
             .select('a.sum, 'b.sum)
 
-    val results = t.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = t.toRetractStream[Row]
+    results.addSink(new StreamITCase.RetractingSink).setParallelism(1)
     env.execute()
 
-    val expected = mutable.MutableList(
-      "1,1", "3,3", "6,5", "10,8", "15,11", "21,14", "28,18", "36,22", "45,26", "55,30", "66,35",
-      "78,40", "91,45", "105,50", "120,55", "136,61", "153,67", "171,73", "190,79", "210,85",
-      "231,91")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = List("231,91")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   @Test
   def testGroupAggregate(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
-    env.setParallelism(1)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.clear
 
@@ -68,15 +64,12 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
       .groupBy('b)
       .select('b, 'a.sum)
 
-    val results = t.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = t.toRetractStream[Row]
+    results.addSink(new StreamITCase.RetractingSink)
     env.execute()
 
-    val expected = mutable.MutableList(
-      "1,1", "2,2", "2,5", "3,4", "3,9", "3,15", "4,7", "4,15",
-      "4,24", "4,34", "5,11", "5,23", "5,36", "5,50", "5,65", "6,16", "6,33", "6,51", "6,70",
-      "6,90", "6,111")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    val expected = List("1,1", "2,5", "3,15", "4,34", "5,65", "6,111")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   @Test
@@ -88,30 +81,22 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
 
     val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
       .groupBy('b)
-      .select('a.sum as 'd, 'b)
-      .groupBy('b, 'd)
-      .select('b)
+      .select('a.count as 'cnt, 'b)
+      .groupBy('cnt)
+      .select('cnt, 'b.count as 'freq)
 
-    val results = t.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
+    val results = t.toRetractStream[Row]
 
-    val expected = mutable.MutableList(
-      "1",
-      "2", "2",
-      "3", "3", "3",
-      "4", "4", "4", "4",
-      "5", "5", "5", "5", "5",
-      "6", "6", "6", "6", "6", "6")
-
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+    results.addSink(new RetractingSink)
+    env.execute()
+    val expected = List("1,1", "2,1", "3,1", "4,1", "5,1", "6,1")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 
   @Test
   def testGroupAggregateWithExpression(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     env.setStateBackend(getStateBackend)
-    env.setParallelism(1)
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.clear
 
@@ -119,14 +104,14 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
       .groupBy('e, 'b % 3)
       .select('c.min, 'e, 'a.avg, 'd.count)
 
-    val results = t.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
+    val results = t.toRetractStream[Row]
+    results.addSink(new RetractingSink)
     env.execute()
 
     val expected = mutable.MutableList(
-      "0,1,1,1", "1,2,2,1", "2,1,2,1", "3,2,3,1", "1,2,2,2",
-      "5,3,3,1", "3,2,3,2", "7,1,4,1", "2,1,3,2", "3,2,3,3", "7,1,4,2", "5,3,4,2", "12,3,5,1",
-      "1,2,3,3", "14,2,5,1")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+      "0,1,1,1", "7,1,4,2", "2,1,3,2",
+      "3,2,3,3", "1,2,3,3", "14,2,5,1",
+      "12,3,5,1", "5,3,4,2")
+    assertEquals(expected.sorted, StreamITCase.retractedResults.sorted)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
index ea3ab22..96e5eb5 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/OverWindowTest.scala
@@ -35,7 +35,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows as 'w)
       .select('c, 'b.count over 'x)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -43,7 +43,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over partitionBy 'c orderBy 'abc preceding 2.rows as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -51,7 +51,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over partitionBy 'c orderBy 'rowtime preceding 2 following "xx" as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -59,7 +59,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows following CURRENT_RANGE as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -67,7 +67,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over partitionBy 'a + 'b orderBy 'rowtime preceding 2.rows as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -77,7 +77,7 @@ class OverWindowTest extends TableTestBase {
     val result = table2
       .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -85,7 +85,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over orderBy 'rowtime preceding -1.rows as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -93,7 +93,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over orderBy 'rowtime preceding 1.rows following -2.rows as 'w)
       .select('c, 'b.count over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -103,7 +103,7 @@ class OverWindowTest extends TableTestBase {
     val result = table
       .window(Over orderBy 'rowtime preceding 1.minutes as 'w)
       .select('c, weightedAvg('b, 'a) over 'w)
-    streamUtil.tEnv.optimize(result.getRelNode)
+    streamUtil.tEnv.optimize(result.getRelNode, updatesAsRetraction = true)
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
index 497869d..effde8e 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
@@ -25,16 +25,18 @@ import org.junit.Assert._
 
 import scala.collection.mutable
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
-import org.apache.flink.table.runtime.types.CRow
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
 
 object StreamITCase {
 
-  var testResults = mutable.MutableList.empty[String]
+  var testResults: mutable.MutableList[String] = mutable.MutableList.empty[String]
+  var retractedResults: ArrayBuffer[String] = mutable.ArrayBuffer.empty[String]
 
   def clear = {
     StreamITCase.testResults.clear()
+    StreamITCase.retractedResults.clear()
   }
 
   def compareWithList(expected: java.util.List[String]): Unit = {
@@ -49,4 +51,33 @@ object StreamITCase {
       }
     }
   }
+
+  final class RetractMessagesSink extends RichSinkFunction[(Boolean, Row)]() {
+    def invoke(v: (Boolean, Row)) {
+      testResults.synchronized {
+        testResults += (if (v._1) "+" else "-") + v._2
+      }
+    }
+  }
+
+  final class RetractingSink() extends RichSinkFunction[(Boolean, Row)] {
+    def invoke(v: (Boolean, Row)) {
+      retractedResults.synchronized {
+        val value = v._2.toString
+        if (v._1) {
+          retractedResults += value
+        } else {
+          val idx = retractedResults.indexOf(value)
+          if (idx >= 0) {
+            retractedResults.remove(idx)
+          } else {
+            throw new RuntimeException("Tried to retract a value that wasn't added first. " +
+              "This is probably an incorrectly implemented test. " +
+              "Try to set the parallelism of the sink to 1.")
+          }
+        }
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
index 580029f..861f70e 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
@@ -292,7 +292,7 @@ class StreamTableTestForRetractionUtil extends StreamTableTestUtil {
 
   def verifyTableTrait(resultTable: Table, expected: String): Unit = {
     val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
+    val optimized = tEnv.optimize(relNode, updatesAsRetraction = false)
     val actual = TraitUtil.toString(optimized)
     assertEquals(
       expected.split("\n").map(_.trim).mkString("\n"),

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sinks/StreamTableSinksITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sinks/StreamTableSinksITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sinks/StreamTableSinksITCase.scala
new file mode 100644
index 0000000..2dfb658
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sinks/StreamTableSinksITCase.scala
@@ -0,0 +1,511 @@
+/*
+ * 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.table.sinks
+
+import java.lang.{Boolean => JBool}
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.functions.sink.SinkFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.StreamTestData
+import org.apache.flink.types.Row
+
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+class StreamTableSinksITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test(expected = classOf[TableException])
+  def testAppendSinkOnUpdatingTable(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'id, 'num, 'text)
+
+    t.groupBy('text)
+      .select('text, 'id.count, 'num.sum)
+      .writeToSink(new TestAppendSink)
+
+    // must fail because table is not append-only
+    env.execute()
+  }
+
+  @Test
+  def testAppendSinkOnAppendTable(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+        .assignAscendingTimestamps(_._1.toLong)
+        .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+
+    t.window(Tumble over 5.millis on 'rowtime as 'w)
+      .groupBy('w)
+      .select('w.end, 'id.count, 'num.sum)
+      .writeToSink(new TestAppendSink)
+
+    env.execute()
+
+    val result = RowCollector.getAndClearValues.map(_.f1.toString).sorted
+    val expected = List(
+      "1970-01-01 00:00:00.005,4,8",
+      "1970-01-01 00:00:00.01,5,18",
+      "1970-01-01 00:00:00.015,5,24",
+      "1970-01-01 00:00:00.02,5,29",
+      "1970-01-01 00:00:00.025,2,12")
+      .sorted
+    assertEquals(expected, result)
+  }
+
+  @Test
+  def testRetractSinkOnUpdatingTable(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text)
+
+    t.select('id, 'num, 'text.charLength() as 'len)
+      .groupBy('len)
+      .select('len, 'id.count, 'num.sum)
+      .writeToSink(new TestRetractSink)
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    val retracted = restractResults(results).sorted
+    val expected = List(
+      "2,1,1",
+      "5,1,2",
+      "11,1,2",
+      "25,1,3",
+      "10,7,39",
+      "14,1,3",
+      "9,9,41").sorted
+    assertEquals(expected, retracted)
+
+  }
+
+  @Test
+  def testRetractSinkOnAppendTable(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+
+    t.window(Tumble over 5.millis on 'rowtime as 'w)
+      .groupBy('w)
+      .select('w.end, 'id.count, 'num.sum)
+      .writeToSink(new TestRetractSink)
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    assertFalse(
+      "Received retraction messages for append only table",
+      results.exists(!_.f0))
+
+    val retracted = restractResults(results).sorted
+    val expected = List(
+      "1970-01-01 00:00:00.005,4,8",
+      "1970-01-01 00:00:00.01,5,18",
+      "1970-01-01 00:00:00.015,5,24",
+      "1970-01-01 00:00:00.02,5,29",
+      "1970-01-01 00:00:00.025,2,12")
+      .sorted
+    assertEquals(expected, retracted)
+
+  }
+
+  @Test
+  def testUpsertSinkOnUpdatingTableWithFullKey(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text)
+
+    t.select('id, 'num, 'text.charLength() as 'len, ('id > 0) as 'cTrue)
+      .groupBy('len, 'cTrue)
+      .select('len, 'id.count as 'cnt, 'cTrue)
+      .groupBy('cnt, 'cTrue)
+      .select('cnt, 'len.count, 'cTrue)
+      .writeToSink(new TestUpsertSink(Array("cnt", "cTrue"), false))
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    assertTrue(
+      "Results must include delete messages",
+      results.exists(_.f0 == false)
+    )
+
+    val retracted = upsertResults(results, Array(0, 2)).sorted
+    val expected = List(
+      "1,5,true",
+      "7,1,true",
+      "9,1,true").sorted
+    assertEquals(expected, retracted)
+
+  }
+
+  @Test(expected = classOf[TableException])
+  def testUpsertSinkOnUpdatingTableWithoutFullKey(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text)
+
+    t.select('id, 'num, 'text.charLength() as 'len, ('id > 0) as 'cTrue)
+      .groupBy('len, 'cTrue)
+      .select('len, 'id.count, 'num.sum)
+      .writeToSink(new TestUpsertSink(Array("len", "cTrue"), false))
+
+    // must fail because table is updating table without full key
+    env.execute()
+  }
+
+  @Test
+  def testUpsertSinkOnAppendingTableWithFullKey1(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+
+    t.window(Tumble over 5.millis on 'rowtime as 'w)
+      .groupBy('w, 'num)
+      .select('num, 'w.end as 'wend, 'id.count)
+      .writeToSink(new TestUpsertSink(Array("wend", "num"), true))
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    assertFalse(
+      "Received retraction messages for append only table",
+      results.exists(!_.f0))
+
+    val retracted = upsertResults(results, Array(0, 1, 2)).sorted
+    val expected = List(
+      "1,1970-01-01 00:00:00.005,1",
+      "2,1970-01-01 00:00:00.005,2",
+      "3,1970-01-01 00:00:00.005,1",
+      "3,1970-01-01 00:00:00.01,2",
+      "4,1970-01-01 00:00:00.01,3",
+      "4,1970-01-01 00:00:00.015,1",
+      "5,1970-01-01 00:00:00.015,4",
+      "5,1970-01-01 00:00:00.02,1",
+      "6,1970-01-01 00:00:00.02,4",
+      "6,1970-01-01 00:00:00.025,2").sorted
+    assertEquals(expected, retracted)
+  }
+
+  @Test
+  def testUpsertSinkOnAppendingTableWithFullKey2(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+
+    t.window(Tumble over 5.millis on 'rowtime as 'w)
+      .groupBy('w, 'num)
+      .select('w.start as 'wstart, 'w.end as 'wend, 'num, 'id.count)
+      .writeToSink(new TestUpsertSink(Array("wstart", "wend", "num"), true))
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    assertFalse(
+      "Received retraction messages for append only table",
+      results.exists(!_.f0))
+
+    val retracted = upsertResults(results, Array(0, 1, 2)).sorted
+    val expected = List(
+      "1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,1,1",
+      "1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,2,2",
+      "1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,3,1",
+      "1970-01-01 00:00:00.005,1970-01-01 00:00:00.01,3,2",
+      "1970-01-01 00:00:00.005,1970-01-01 00:00:00.01,4,3",
+      "1970-01-01 00:00:00.01,1970-01-01 00:00:00.015,4,1",
+      "1970-01-01 00:00:00.01,1970-01-01 00:00:00.015,5,4",
+      "1970-01-01 00:00:00.015,1970-01-01 00:00:00.02,5,1",
+      "1970-01-01 00:00:00.015,1970-01-01 00:00:00.02,6,4",
+      "1970-01-01 00:00:00.02,1970-01-01 00:00:00.025,6,2").sorted
+    assertEquals(expected, retracted)
+  }
+
+  @Test
+  def testUpsertSinkOnAppendingTableWithoutFullKey1(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+
+    t.window(Tumble over 5.millis on 'rowtime as 'w)
+      .groupBy('w, 'num)
+      .select('w.end as 'wend, 'id.count as 'cnt)
+      .writeToSink(new TestUpsertSink(null, true))
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    assertFalse(
+      "Received retraction messages for append only table",
+      results.exists(!_.f0))
+
+    val retracted = results.map(_.f1.toString).sorted
+    val expected = List(
+      "1970-01-01 00:00:00.005,1",
+      "1970-01-01 00:00:00.005,2",
+      "1970-01-01 00:00:00.005,1",
+      "1970-01-01 00:00:00.01,2",
+      "1970-01-01 00:00:00.01,3",
+      "1970-01-01 00:00:00.015,1",
+      "1970-01-01 00:00:00.015,4",
+      "1970-01-01 00:00:00.02,1",
+      "1970-01-01 00:00:00.02,4",
+      "1970-01-01 00:00:00.025,2").sorted
+    assertEquals(expected, retracted)
+  }
+
+  @Test
+  def testUpsertSinkOnAppendingTableWithoutFullKey2(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val t = StreamTestData.get3TupleDataStream(env)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+
+    t.window(Tumble over 5.millis on 'rowtime as 'w)
+      .groupBy('w, 'num)
+      .select('num, 'id.count as 'cnt)
+      .writeToSink(new TestUpsertSink(null, true))
+
+    env.execute()
+    val results = RowCollector.getAndClearValues
+
+    assertFalse(
+      "Received retraction messages for append only table",
+      results.exists(!_.f0))
+
+    val retracted = results.map(_.f1.toString).sorted
+    val expected = List(
+      "1,1",
+      "2,2",
+      "3,1",
+      "3,2",
+      "4,3",
+      "4,1",
+      "5,4",
+      "5,1",
+      "6,4",
+      "6,2").sorted
+    assertEquals(expected, retracted)
+  }
+
+  /** Converts a list of retraction messages into a list of final results. */
+  private def restractResults(results: List[JTuple2[JBool, Row]]): List[String] = {
+
+    val retracted = results
+      .foldLeft(Map[String, Int]()){ (m: Map[String, Int], v: JTuple2[JBool, Row]) =>
+        val cnt = m.getOrElse(v.f1.toString, 0)
+        if (v.f0) {
+          m + (v.f1.toString -> (cnt + 1))
+        } else {
+          m + (v.f1.toString -> (cnt - 1))
+        }
+      }.filter{ case (_, c: Int) => c != 0 }
+
+    assertFalse(
+      "Received retracted rows which have not been accumulated.",
+      retracted.exists{ case (_, c: Int) => c < 0})
+
+    retracted.flatMap { case (r: String, c: Int) => (0 until c).map(_ => r) }.toList
+  }
+
+  /** Converts a list of upsert messages into a list of final results. */
+  private def upsertResults(results: List[JTuple2[JBool, Row]], keys: Array[Int]): List[String] = {
+
+    def getKeys(r: Row): List[String] =
+      keys.foldLeft(List[String]())((k, i) => r.getField(i).toString :: k)
+
+    val upserted = results.foldLeft(Map[String, String]()){ (o: Map[String, String], r) =>
+      val key = getKeys(r.f1).mkString("")
+      if (r.f0) {
+        o + (key -> r.f1.toString)
+      } else {
+        o - key
+      }
+    }
+
+    upserted.values.toList
+  }
+
+}
+
+private class TestAppendSink extends AppendStreamTableSink[Row] {
+
+  var fNames: Array[String] = _
+  var fTypes: Array[TypeInformation[_]] = _
+
+  override def emitDataStream(s: DataStream[Row]): Unit = {
+    s.map(
+      new MapFunction[Row, JTuple2[JBool, Row]] {
+        override def map(value: Row): JTuple2[JBool, Row] = new JTuple2(true, value)
+      })
+      .addSink(new RowSink)
+  }
+
+  override def getOutputType: TypeInformation[Row] = new RowTypeInfo(fTypes, fNames)
+
+  override def getFieldNames: Array[String] = fNames
+
+  override def getFieldTypes: Array[TypeInformation[_]] = fTypes
+
+  override def configure(
+    fieldNames: Array[String],
+    fieldTypes: Array[TypeInformation[_]]): TableSink[Row] = {
+    val copy = new TestAppendSink
+    copy.fNames = fieldNames
+    copy.fTypes = fieldTypes
+    copy
+  }
+}
+
+private class TestRetractSink extends RetractStreamTableSink[Row] {
+
+  var fNames: Array[String] = _
+  var fTypes: Array[TypeInformation[_]] = _
+
+  override def emitDataStream(s: DataStream[JTuple2[JBool, Row]]): Unit = {
+    s.addSink(new RowSink)
+  }
+
+  override def getRecordType: TypeInformation[Row] = new RowTypeInfo(fTypes, fNames)
+
+  override def getFieldNames: Array[String] = fNames
+
+  override def getFieldTypes: Array[TypeInformation[_]] = fTypes
+
+  override def configure(
+      fieldNames: Array[String],
+      fieldTypes: Array[TypeInformation[_]]): TableSink[JTuple2[JBool, Row]] = {
+    val copy = new TestRetractSink
+    copy.fNames = fieldNames
+    copy.fTypes = fieldTypes
+    copy
+  }
+
+}
+
+private class TestUpsertSink(
+    expectedKeys: Array[String],
+    expectedIsAppendOnly: Boolean)
+  extends UpsertStreamTableSink[Row] {
+
+  var fNames: Array[String] = _
+  var fTypes: Array[TypeInformation[_]] = _
+
+  override def setKeyFields(keys: Array[String]): Unit =
+    if (keys != null) {
+      assertEquals("Provided key fields do not match expected keys",
+        expectedKeys.sorted.mkString(","),
+        keys.sorted.mkString(","))
+    } else {
+      assertNull("Provided key fields should not be null.", expectedKeys)
+    }
+
+  override def setIsAppendOnly(isAppendOnly: Boolean): Unit =
+    assertEquals(
+      "Provided isAppendOnly does not match expected isAppendOnly",
+      expectedIsAppendOnly,
+      isAppendOnly)
+
+  override def getRecordType: TypeInformation[Row] = new RowTypeInfo(fTypes, fNames)
+
+  override def emitDataStream(s: DataStream[JTuple2[JBool, Row]]): Unit = {
+    s.addSink(new RowSink)
+  }
+
+  override def getFieldNames: Array[String] = fNames
+
+  override def getFieldTypes: Array[TypeInformation[_]] = fTypes
+
+  override def configure(
+      fieldNames: Array[String],
+      fieldTypes: Array[TypeInformation[_]]): TableSink[JTuple2[JBool, Row]] = {
+    val copy = new TestUpsertSink(expectedKeys, expectedIsAppendOnly)
+    copy.fNames = fieldNames
+    copy.fTypes = fieldTypes
+    copy
+  }
+}
+
+class RowSink extends SinkFunction[JTuple2[JBool, Row]] {
+  override def invoke(value: JTuple2[JBool, Row]): Unit = RowCollector.addValue(value)
+}
+
+object RowCollector {
+  private val sink: mutable.ArrayBuffer[JTuple2[JBool, Row]] =
+    new mutable.ArrayBuffer[JTuple2[JBool, Row]]()
+
+  def addValue(value: JTuple2[JBool, Row]): Unit = {
+    sink.synchronized {
+      sink += value
+    }
+  }
+
+  def getAndClearValues: List[JTuple2[JBool, Row]] = {
+    val out = sink.toList
+    sink.clear()
+    out
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
index 79e957a..8626b07 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
@@ -39,9 +39,4 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) {
 
   override protected def getBuiltInPhysicalOptRuleSet: RuleSet = ???
 
-  override protected def getConversionMapper[IN, OUT](
-      physicalTypeInfo: TypeInformation[IN],
-      logicalRowType: RelDataType,
-      requestedTypeInfo: TypeInformation[OUT],
-      functionName: String) = ???
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
index 73bc2f8..0e6d461 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
@@ -211,7 +211,7 @@ case class StreamTableTestUtil() extends TableTestUtil {
 
   def verifyTable(resultTable: Table, expected: String): Unit = {
     val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
+    val optimized = tEnv.optimize(relNode, updatesAsRetraction = false)
     val actual = RelOptUtil.toString(optimized)
     assertEquals(
       expected.split("\n").map(_.trim).mkString("\n"),
@@ -221,7 +221,7 @@ case class StreamTableTestUtil() extends TableTestUtil {
   // the print methods are for debugging purposes only
   def printTable(resultTable: Table): Unit = {
     val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
+    val optimized = tEnv.optimize(relNode, updatesAsRetraction = false)
     println(RelOptUtil.toString(optimized))
   }
 


[15/15] flink git commit: [FLINK-6093] [table] Add stream TableSinks and DataStream conversion with support for retraction.

Posted by fh...@apache.org.
[FLINK-6093] [table] Add stream TableSinks and DataStream conversion with support for retraction.


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

Branch: refs/heads/master
Commit: f37988c19adc30d324cde83c54f2fa5d36efb9e7
Parents: bfed279
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Apr 28 01:59:57 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:55 2017 +0200

----------------------------------------------------------------------
 .../connectors/kafka/KafkaTableSink.java        |   6 +-
 .../flink/table/api/BatchTableEnvironment.scala |  16 +-
 .../table/api/StreamTableEnvironment.scala      | 428 +++++++++++-----
 .../apache/flink/table/api/TableConfig.scala    |  17 -
 .../flink/table/api/TableEnvironment.scala      |  48 +-
 .../table/api/java/StreamTableEnvironment.scala |  81 ++-
 .../api/scala/StreamTableEnvironment.scala      |  33 +-
 .../table/api/scala/TableConversions.scala      |  19 +
 .../org/apache/flink/table/api/table.scala      |   4 +-
 .../plan/nodes/datastream/DataStreamCalc.scala  |   8 +-
 .../datastream/DataStreamGroupAggregate.scala   |   2 +
 .../DataStreamGroupWindowAggregate.scala        |   4 +
 .../nodes/datastream/retractionTraits.scala     |  37 +-
 .../datastream/DataStreamRetractionRules.scala  |  16 +-
 .../runtime/CRowCorrelateFlatMapRunner.scala    |   2 +-
 .../flink/table/runtime/CRowFlatMapRunner.scala |   2 +-
 .../table/runtime/CRowInputMapRunner.scala      |   2 +-
 .../runtime/CRowInputTupleOutputMapRunner.scala |  53 +-
 .../table/runtime/CRowOutputMapRunner.scala     |   2 +-
 .../table/runtime/CorrelateFlatMapRunner.scala  |   2 +-
 .../flink/table/runtime/FlatJoinRunner.scala    |   2 +-
 .../flink/table/runtime/FlatMapRunner.scala     |   2 +-
 .../flink/table/runtime/MapJoinLeftRunner.scala |   2 +-
 .../table/runtime/MapJoinRightRunner.scala      |   2 +-
 .../apache/flink/table/runtime/MapRunner.scala  |   2 +-
 .../flink/table/runtime/MapSideJoinRunner.scala |   2 +-
 ...aSetSessionWindowAggregatePreProcessor.scala |   2 +-
 .../aggregate/GroupAggProcessFunction.scala     |  55 +-
 .../runtime/io/CRowValuesInputFormat.scala      |   2 +-
 .../table/runtime/io/ValuesInputFormat.scala    |   2 +-
 .../table/sinks/AppendStreamTableSink.scala     |  36 ++
 .../apache/flink/table/sinks/CsvTableSink.scala | 100 +---
 .../table/sinks/RetractStreamTableSink.scala    |  55 ++
 .../flink/table/sinks/StreamRetractSink.scala   |  35 --
 .../flink/table/sinks/StreamTableSink.scala     |  32 --
 .../table/sinks/UpsertStreamTableSink.scala     |  79 +++
 .../flink/table/TableEnvironmentTest.scala      |   1 -
 .../api/scala/stream/RetractionITCase.scala     |  80 ++-
 .../api/scala/stream/TableSinkITCase.scala      |  33 +-
 .../table/api/scala/stream/sql/SqlITCase.scala  | 102 ++--
 .../stream/table/GroupAggregationsITCase.scala  |  61 +--
 .../api/scala/stream/table/OverWindowTest.scala |  18 +-
 .../api/scala/stream/utils/StreamITCase.scala   |  35 +-
 .../table/plan/rules/RetractionRulesTest.scala  |   2 +-
 .../table/sinks/StreamTableSinksITCase.scala    | 511 +++++++++++++++++++
 .../table/utils/MockTableEnvironment.scala      |   5 -
 .../flink/table/utils/TableTestBase.scala       |   4 +-
 47 files changed, 1433 insertions(+), 611 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
index 97f5fba..a8a2fd0 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.table.sinks.StreamTableSink;
+import org.apache.flink.table.sinks.AppendStreamTableSink;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.SerializationSchema;
@@ -29,12 +29,12 @@ import org.apache.flink.util.Preconditions;
 import java.util.Properties;
 
 /**
- * A version-agnostic Kafka {@link StreamTableSink}.
+ * A version-agnostic Kafka {@link AppendStreamTableSink}.
  *
  * <p>The version-specific Kafka consumers need to extend this class and
  * override {@link #createKafkaProducer(String, Properties, SerializationSchema, KafkaPartitioner)}}.
  */
-public abstract class KafkaTableSink implements StreamTableSink<Row> {
+public abstract class KafkaTableSink implements AppendStreamTableSink<Row> {
 
 	protected final String topic;
 	protected final Properties properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index c7bacfe..2a3cedf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -36,7 +36,7 @@ import org.apache.flink.table.expressions.{Expression, RowtimeAttribute, TimeAtt
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.dataset.DataSetRel
 import org.apache.flink.table.plan.rules.FlinkRuleSets
-import org.apache.flink.table.plan.schema.{DataSetTable, TableSourceTable}
+import org.apache.flink.table.plan.schema.{DataSetTable, RowSchema, TableSourceTable}
 import org.apache.flink.table.runtime.MapRunner
 import org.apache.flink.table.sinks.{BatchTableSink, TableSink}
 import org.apache.flink.table.sources.{BatchTableSource, TableSource}
@@ -133,14 +133,14 @@ abstract class BatchTableEnvironment(
     * Creates a final converter that maps the internal row type to external type.
     *
     * @param physicalTypeInfo the input of the sink
-    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
+    * @param schema the input schema with correct field names (esp. for POJO field mapping)
     * @param requestedTypeInfo the output type of the sink
     * @param functionName name of the map function. Must not be unique but has to be a
     *                     valid Java class identifier.
     */
-  override protected def getConversionMapper[IN, OUT](
+  protected def getConversionMapper[IN, OUT](
       physicalTypeInfo: TypeInformation[IN],
-      logicalRowType: RelDataType,
+      schema: RowSchema,
       requestedTypeInfo: TypeInformation[OUT],
       functionName: String):
     Option[MapFunction[IN, OUT]] = {
@@ -153,7 +153,7 @@ abstract class BatchTableEnvironment(
 
       val converterFunction = generateRowConverterFunction[OUT](
         physicalTypeInfo.asInstanceOf[TypeInformation[Row]],
-        logicalRowType,
+        schema,
         requestedTypeInfo,
         functionName
       )
@@ -334,7 +334,11 @@ abstract class BatchTableEnvironment(
       case node: DataSetRel =>
         val plan = node.translateToPlan(this)
         val conversion =
-          getConversionMapper(plan.getType, logicalType, tpe, "DataSetSinkConversion")
+          getConversionMapper(
+            plan.getType,
+            new RowSchema(logicalType),
+            tpe,
+            "DataSetSinkConversion")
         conversion match {
           case None => plan.asInstanceOf[DataSet[A]] // no conversion necessary
           case Some(mapFunction: MapFunction[Row, A]) =>

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index bd06305..aef2b1b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -19,19 +19,23 @@
 package org.apache.flink.table.api
 
 import _root_.java.util.concurrent.atomic.AtomicInteger
+import _root_.java.lang.{Boolean => JBool}
 
 import org.apache.calcite.plan.RelOptUtil
 import org.apache.calcite.plan.hep.HepMatchOrder
-import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.{RelNode, RelVisitor}
 import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode}
+import org.apache.calcite.sql.SqlKind
 import org.apache.calcite.sql2rel.RelDecorrelator
 import org.apache.calcite.tools.{RuleSet, RuleSets}
 import org.apache.flink.api.common.typeinfo.AtomicType
 import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
-import org.apache.flink.api.java.typeutils.{GenericTypeInfo, TupleTypeInfo}
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.table.calcite.RelTimeIndicatorConverter
@@ -39,12 +43,12 @@ import org.apache.flink.table.explain.PlanJsonParser
 import org.apache.flink.table.expressions.{Expression, ProctimeAttribute, RowtimeAttribute, UnresolvedFieldReference}
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.{DataStreamRel, UpdateAsRetractionTrait}
+import org.apache.flink.table.plan.nodes.datastream._
 import org.apache.flink.table.plan.rules.FlinkRuleSets
-import org.apache.flink.table.plan.schema.StreamTableSourceTable
-import org.apache.flink.table.plan.schema.DataStreamTable
-import org.apache.flink.table.runtime.{CRowInputMapRunner, CRowInputTupleOutputMapRunner}
+import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema, StreamTableSourceTable}
+import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
-import org.apache.flink.table.sinks.{StreamRetractSink, StreamTableSink, TableSink}
+import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink}
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
 import org.apache.flink.table.typeutils.TypeCheckUtils
 import org.apache.flink.types.Row
@@ -127,97 +131,192 @@ abstract class StreamTableEnvironment(
   override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
 
     sink match {
-      case streamSink: StreamTableSink[T] =>
+
+      case retractSink: RetractStreamTableSink[_] =>
+        // retraction sink can always be used
         val outputType = sink.getOutputType
         // translate the Table into a DataStream and provide the type that the TableSink expects.
-        val result: DataStream[T] = translate(table)(outputType)
-        // Give the DataSet to the TableSink to emit it.
-        streamSink.emitDataStream(result)
-
-      case streamRetractSink: StreamRetractSink[T] =>
+        val result: DataStream[T] =
+          translate(table, updatesAsRetraction = true, withChangeFlag = true)(outputType)
+        // Give the DataStream to the TableSink to emit it.
+        retractSink.asInstanceOf[RetractStreamTableSink[Any]]
+          .emitDataStream(result.asInstanceOf[DataStream[JTuple2[JBool, Any]]])
+
+      case upsertSink: UpsertStreamTableSink[_] =>
+        // optimize plan
+        val optimizedPlan = optimize(table.getRelNode, updatesAsRetraction = false)
+        // check for append only table
+        val isAppendOnlyTable = isAppendOnly(optimizedPlan)
+        upsertSink.setIsAppendOnly(isAppendOnlyTable)
+        // extract unique key fields
+        val tableKeys: Option[Array[String]] = getUniqueKeyFields(optimizedPlan)
+        // check that we have keys if the table has changes (is not append-only)
+        tableKeys match {
+          case Some(keys) => upsertSink.setKeyFields(keys)
+          case None if isAppendOnlyTable => upsertSink.setKeyFields(null)
+          case None if !isAppendOnlyTable => throw new TableException(
+            "UpsertStreamTableSink requires that Table has a full primary keys if it is updated.")
+        }
+        val outputType = sink.getOutputType
+        // translate the Table into a DataStream and provide the type that the TableSink expects.
+        val result: DataStream[T] =
+          translate(optimizedPlan, table.getRelNode.getRowType, withChangeFlag = true)(outputType)
+        // Give the DataStream to the TableSink to emit it.
+        upsertSink.asInstanceOf[UpsertStreamTableSink[Any]]
+          .emitDataStream(result.asInstanceOf[DataStream[JTuple2[JBool, Any]]])
+
+      case appendSink: AppendStreamTableSink[_] =>
+        // optimize plan
+        val optimizedPlan = optimize(table.getRelNode, updatesAsRetraction = false)
+        // verify table is an insert-only (append-only) table
+        if (!isAppendOnly(optimizedPlan)) {
+          throw new TableException(
+            "AppendStreamTableSink requires that Table has only insert changes.")
+        }
         val outputType = sink.getOutputType
-        this.config.setNeedsUpdatesAsRetractionForSink(streamRetractSink.needsUpdatesAsRetraction)
         // translate the Table into a DataStream and provide the type that the TableSink expects.
-        val result: DataStream[JTuple2[Boolean, T]] = translate(table, true)(outputType)
-        // Give the DataSet to the TableSink to emit it.
-        streamRetractSink.emitDataStreamWithChange(result)
+        val result: DataStream[T] =
+          translate(optimizedPlan, table.getRelNode.getRowType, withChangeFlag = false)(outputType)
+        // Give the DataStream to the TableSink to emit it.
+        appendSink.asInstanceOf[AppendStreamTableSink[T]].emitDataStream(result)
+
       case _ =>
-        throw new TableException("StreamTableSink required to emit streaming Table")
+        throw new TableException("Stream Tables can only be emitted by AppendStreamTableSink, " +
+          "RetractStreamTableSink, or UpsertStreamTableSink.")
     }
   }
 
-
   /**
     * Creates a final converter that maps the internal row type to external type.
     *
     * @param physicalTypeInfo the input of the sink
-    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
+    * @param schema the input schema with correct field names (esp. for POJO field mapping)
     * @param requestedTypeInfo the output type of the sink
     * @param functionName name of the map function. Must not be unique but has to be a
     *                     valid Java class identifier.
     */
-  override protected def getConversionMapper[IN, OUT](
+  protected def getConversionMapper[IN, OUT](
       physicalTypeInfo: TypeInformation[IN],
-      logicalRowType: RelDataType,
+      schema: RowSchema,
       requestedTypeInfo: TypeInformation[OUT],
       functionName: String):
-  Option[MapFunction[IN, OUT]] = {
+    MapFunction[IN, OUT] = {
 
-    if (requestedTypeInfo.getTypeClass == classOf[CRow]) {
-      // only used to explain table
-      None
-    } else if (requestedTypeInfo.getTypeClass == classOf[Row]) {
+    if (requestedTypeInfo.getTypeClass == classOf[Row]) {
       // CRow to Row, only needs to be unwrapped
-      Some(
-        new MapFunction[CRow, Row] {
-          override def map(value: CRow): Row = value.row
-        }.asInstanceOf[MapFunction[IN, OUT]]
-      )
+      new MapFunction[CRow, Row] {
+        override def map(value: CRow): Row = value.row
+      }.asInstanceOf[MapFunction[IN, OUT]]
     } else {
       // Some type that is neither CRow nor Row
       val converterFunction = generateRowConverterFunction[OUT](
         physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
-        logicalRowType,
+        schema,
         requestedTypeInfo,
         functionName
       )
 
-      Some(new CRowInputMapRunner[OUT](
+      new CRowInputMapRunner[OUT](
         converterFunction.name,
         converterFunction.code,
         converterFunction.returnType)
-        .asInstanceOf[MapFunction[IN, OUT]])
+        .asInstanceOf[MapFunction[IN, OUT]]
     }
   }
 
+  /** Validates that the plan produces only append changes. */
+  protected def isAppendOnly(plan: RelNode): Boolean = {
+    val appendOnlyValidator = new AppendOnlyValidator
+    appendOnlyValidator.go(plan)
+
+    appendOnlyValidator.isAppendOnly
+  }
+
+  /** Extracts the unique keys of the table produced by the plan. */
+  protected def getUniqueKeyFields(plan: RelNode): Option[Array[String]] = {
+    val keyExtractor = new UniqueKeyExtractor
+    keyExtractor.go(plan)
+    keyExtractor.keys
+  }
+
   /**
-    * Creates a final converter that maps the internal CRow type to external Tuple2 type.
+    * Creates a converter that maps the internal CRow type to Scala or Java Tuple2 with change flag.
     *
     * @param physicalTypeInfo the input of the sink
-    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
-    * @param requestedTypeInfo the output type of the sink
+    * @param schema the input schema with correct field names (esp. for POJO field mapping)
+    * @param requestedTypeInfo the output type of the sink.
     * @param functionName name of the map function. Must not be unique but has to be a
     *                     valid Java class identifier.
     */
-  protected def getTupleConversionMapper[IN, OUT](
-      physicalTypeInfo: TypeInformation[IN],
-      logicalRowType: RelDataType,
-      requestedTypeInfo: TypeInformation[OUT],
-      functionName: String):
-  Option[MapFunction[IN, JTuple2[Boolean, OUT]]] = {
+  private def getConversionMapperWithChanges[OUT](
+    physicalTypeInfo: TypeInformation[CRow],
+    schema: RowSchema,
+    requestedTypeInfo: TypeInformation[OUT],
+    functionName: String):
+  MapFunction[CRow, OUT] = {
+
+    requestedTypeInfo match {
+
+      // Scala tuple
+      case t: CaseClassTypeInfo[_]
+        if t.getTypeClass == classOf[(_, _)] && t.getTypeAt(0) == Types.BOOLEAN =>
+
+        val reqType = t.getTypeAt(1).asInstanceOf[TypeInformation[Any]]
+        if (reqType.getTypeClass == classOf[Row]) {
+          // Requested type is Row. Just rewrap CRow in Tuple2
+          new MapFunction[CRow, (Boolean, Row)] {
+            override def map(cRow: CRow): (Boolean, Row) = {
+              (cRow.change, cRow.row)
+            }
+          }.asInstanceOf[MapFunction[CRow, OUT]]
+        } else {
+          // Use a map function to convert Row into requested type and wrap result in Tuple2
+          val converterFunction = generateRowConverterFunction(
+            physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
+            schema,
+            reqType,
+            functionName
+          )
+
+          new CRowInputScalaTupleOutputMapRunner(
+            converterFunction.name,
+            converterFunction.code,
+            requestedTypeInfo.asInstanceOf[TypeInformation[(Boolean, Any)]])
+            .asInstanceOf[MapFunction[CRow, OUT]]
 
-    val converterFunction = generateRowConverterFunction(
-      physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
-      logicalRowType,
-      requestedTypeInfo,
-      functionName
-    )
+        }
 
-    Some(new CRowInputTupleOutputMapRunner[OUT](
-      converterFunction.name,
-      converterFunction.code,
-      new TupleTypeInfo(BasicTypeInfo.BOOLEAN_TYPE_INFO, requestedTypeInfo))
-           .asInstanceOf[MapFunction[IN, JTuple2[Boolean, OUT]]])
+      // Java tuple
+      case t: TupleTypeInfo[_]
+        if t.getTypeClass == classOf[JTuple2[_, _]] && t.getTypeAt(0) == Types.BOOLEAN =>
+
+        val reqType = t.getTypeAt(1).asInstanceOf[TypeInformation[Any]]
+        if (reqType.getTypeClass == classOf[Row]) {
+          // Requested type is Row. Just rewrap CRow in Tuple2
+          new MapFunction[CRow, JTuple2[JBool, Row]] {
+            val outT = new JTuple2(true.asInstanceOf[JBool], null.asInstanceOf[Row])
+            override def map(cRow: CRow): JTuple2[JBool, Row] = {
+              outT.f0 = cRow.change
+              outT.f1 = cRow.row
+              outT
+            }
+          }.asInstanceOf[MapFunction[CRow, OUT]]
+        } else {
+          // Use a map function to convert Row into requested type and wrap result in Tuple2
+          val converterFunction = generateRowConverterFunction(
+            physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
+            schema,
+            reqType,
+            functionName
+          )
+
+          new CRowInputJavaTupleOutputMapRunner(
+            converterFunction.name,
+            converterFunction.code,
+            requestedTypeInfo.asInstanceOf[TypeInformation[JTuple2[JBool, Any]]])
+            .asInstanceOf[MapFunction[CRow, OUT]]
+        }
+    }
   }
 
   /**
@@ -380,9 +479,10 @@ abstract class StreamTableEnvironment(
     * Generates the optimized [[RelNode]] tree from the original relational node tree.
     *
     * @param relNode The root node of the relational expression tree.
+    * @param updatesAsRetraction True if the sink requests updates as retraction messages.
     * @return The optimized [[RelNode]] tree
     */
-  private[flink] def optimize(relNode: RelNode): RelNode = {
+  private[flink] def optimize(relNode: RelNode, updatesAsRetraction: Boolean): RelNode = {
 
     // 1. decorrelate
     val decorPlan = RelDecorrelator.decorrelateQuery(relNode)
@@ -410,7 +510,7 @@ abstract class StreamTableEnvironment(
     // 5. optimize the physical Flink plan
     val physicalOptRuleSet = getPhysicalOptRuleSet
     val physicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.DATASTREAM).simplify()
-    var physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
+    val physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
       runVolcanoPlanner(physicalOptRuleSet, logicalPlan, physicalOutputProps)
     } else {
       logicalPlan
@@ -419,13 +519,18 @@ abstract class StreamTableEnvironment(
     // 6. decorate the optimized plan
     val decoRuleSet = getDecoRuleSet
     val decoratedPlan = if (decoRuleSet.iterator().hasNext) {
-
-      if (this.config.getNeedsUpdatesAsRetractionForSink) {
-        physicalPlan = physicalPlan.copy(
+      val planToDecorate = if (updatesAsRetraction) {
+        physicalPlan.copy(
           physicalPlan.getTraitSet.plus(new UpdateAsRetractionTrait(true)),
           physicalPlan.getInputs)
+      } else {
+        physicalPlan
       }
-      runHepPlanner(HepMatchOrder.BOTTOM_UP, decoRuleSet, physicalPlan, physicalPlan.getTraitSet)
+      runHepPlanner(
+        HepMatchOrder.BOTTOM_UP,
+        decoRuleSet,
+        planToDecorate,
+        planToDecorate.getTraitSet)
     } else {
       physicalPlan
     }
@@ -440,14 +545,17 @@ abstract class StreamTableEnvironment(
     * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators.
     *
     * @param table The root node of the relational expression tree.
+    * @param updatesAsRetraction Set to true to encode updates as retraction messages.
+    * @param withChangeFlag Set to true to emit records with change flags.
     * @param tpe The [[TypeInformation]] of the resulting [[DataStream]].
     * @tparam A The type of the resulting [[DataStream]].
     * @return The [[DataStream]] that corresponds to the translated [[Table]].
     */
-  protected def translate[A](table: Table)(implicit tpe: TypeInformation[A]): DataStream[A] = {
+  protected def translate[A](table: Table, updatesAsRetraction: Boolean, withChangeFlag: Boolean)
+      (implicit tpe: TypeInformation[A]): DataStream[A] = {
     val relNode = table.getRelNode
-    val dataStreamPlan = optimize(relNode)
-    translate(dataStreamPlan, relNode.getRowType)
+    val dataStreamPlan = optimize(relNode, updatesAsRetraction)
+    translate(dataStreamPlan, relNode.getRowType, withChangeFlag)
   }
 
   /**
@@ -456,87 +564,65 @@ abstract class StreamTableEnvironment(
     * @param logicalPlan The root node of the relational expression tree.
     * @param logicalType The row type of the result. Since the logicalPlan can lose the
     *                    field naming during optimization we pass the row type separately.
+    * @param withChangeFlag Set to true to emit records with change flags.
     * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
     * @tparam A The type of the resulting [[DataStream]].
     * @return The [[DataStream]] that corresponds to the translated [[Table]].
     */
   protected def translate[A](
       logicalPlan: RelNode,
-      logicalType: RelDataType)
+      logicalType: RelDataType,
+      withChangeFlag: Boolean)
       (implicit tpe: TypeInformation[A]): DataStream[A] = {
 
-    TableEnvironment.validateType(tpe)
+    // if no change flags are requested, verify table is an insert-only (append-only) table.
+    if (!withChangeFlag && !isAppendOnly(logicalPlan)) {
+      throw new TableException(
+        "Table is not an append-only table. " +
+          "Output needs to handle update and delete changes.")
+    }
 
-    logicalPlan match {
-      case node: DataStreamRel =>
-        val plan = node.translateToPlan(this)
-        val conversion =
-          getConversionMapper(plan.getType, logicalType, tpe, "DataStreamSinkConversion")
-        conversion match {
-          case None => plan.asInstanceOf[DataStream[A]] // no conversion necessary
-          case Some(mapFunction: MapFunction[CRow, A]) =>
-            plan.map(mapFunction)
-              .returns(tpe)
-              .name(s"to: ${tpe.getTypeClass.getSimpleName}")
-              .asInstanceOf[DataStream[A]]
-        }
+    // get CRow plan
+    val plan: DataStream[CRow] = translateToCRow(logicalPlan)
 
-      case _ =>
-        throw TableException("Cannot generate DataStream due to an invalid logical plan. " +
-          "This is a bug and should not happen. Please file an issue.")
+    // convert CRow to output type
+    val conversion = if (withChangeFlag) {
+      getConversionMapperWithChanges(
+        plan.getType,
+        new RowSchema(logicalType),
+        tpe,
+        "DataStreamSinkConversion")
+    } else {
+      getConversionMapper(
+        plan.getType,
+        new RowSchema(logicalType),
+        tpe,
+        "DataStreamSinkConversion")
     }
-  }
 
-  /**
-    * Translates a [[Table]] into a [[DataStream]] with change information.
-    *
-    * The transformation involves optimizing the relational expression tree as defined by
-    * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators.
-    *
-    * @param table       The root node of the relational expression tree.
-    * @param wrapToTuple True, if want to output chang information
-    * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
-    * @tparam A The type of the resulting [[DataStream]].
-    * @return The [[DataStream]] that corresponds to the translated [[Table]].
-    */
-  protected def translate[A](table: Table, wrapToTuple: Boolean)(implicit tpe: TypeInformation[A])
-  : DataStream[JTuple2[Boolean, A]] = {
-    val relNode = table.getRelNode
-    val dataStreamPlan = optimize(relNode)
-    translate(dataStreamPlan, relNode.getRowType, wrapToTuple)
+    val rootParallelism = plan.getParallelism
+
+    conversion match {
+      case mapFunction: MapFunction[CRow, A] =>
+        plan.map(mapFunction)
+          .returns(tpe)
+          .name(s"to: ${tpe.getTypeClass.getSimpleName}")
+          .setParallelism(rootParallelism)
+    }
   }
 
   /**
-    * Translates a logical [[RelNode]] into a [[DataStream]] with change information.
+    * Translates a logical [[RelNode]] plan into a [[DataStream]] of type [[CRow]].
     *
-    * @param logicalPlan The root node of the relational expression tree.
-    * @param logicalType The row type of the result. Since the logicalPlan can lose the
-    * @param wrapToTuple True, if want to output chang information
-    * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
-    * @tparam A The type of the resulting [[DataStream]].
-    * @return The [[DataStream]] that corresponds to the translated [[Table]].
+    * @param logicalPlan The logical plan to translate.
+    * @return The [[DataStream]] of type [[CRow]].
     */
-  protected def translate[A](
-      logicalPlan: RelNode,
-      logicalType: RelDataType,
-      wrapToTuple: Boolean)(implicit tpe: TypeInformation[A]): DataStream[JTuple2[Boolean, A]] = {
-
-    TableEnvironment.validateType(tpe)
+  protected def translateToCRow(
+    logicalPlan: RelNode): DataStream[CRow] = {
 
     logicalPlan match {
       case node: DataStreamRel =>
-        val plan = node.translateToPlan(this)
-        val conversion =
-          getTupleConversionMapper(plan.getType, logicalType, tpe, "DataStreamSinkConversion")
-        conversion match {
-          case None => plan.asInstanceOf[DataStream[JTuple2[Boolean, A]]] // no conversion necessary
-          case Some(mapFunction: MapFunction[CRow, JTuple2[Boolean, A]]) =>
-            plan.map(mapFunction)
-              .returns(new TupleTypeInfo[JTuple2[Boolean, A]](BasicTypeInfo.BOOLEAN_TYPE_INFO, tpe))
-              .name(s"to: ${tpe.getTypeClass.getSimpleName}")
-              .asInstanceOf[DataStream[JTuple2[Boolean, A]]]
-        }
-
+        node.translateToPlan(this)
       case _ =>
         throw TableException("Cannot generate DataStream due to an invalid logical plan. " +
           "This is a bug and should not happen. Please file an issue.")
@@ -551,10 +637,8 @@ abstract class StreamTableEnvironment(
     */
   def explain(table: Table): String = {
     val ast = table.getRelNode
-    val optimizedPlan = optimize(ast)
-    val dataStream = translate[CRow](
-      optimizedPlan,
-      ast.getRowType)(new GenericTypeInfo(classOf[CRow]))
+    val optimizedPlan = optimize(ast, updatesAsRetraction = false)
+    val dataStream = translateToCRow(optimizedPlan)
 
     val env = dataStream.getExecutionEnvironment
     val jsonSqlPlan = env.getExecutionPlan
@@ -574,4 +658,90 @@ abstract class StreamTableEnvironment(
         s"$sqlPlan"
   }
 
+  private class AppendOnlyValidator extends RelVisitor {
+
+    var isAppendOnly = true
+
+    override def visit(node: RelNode, ordinal: Int, parent: RelNode): Unit = {
+      node match {
+        case s: DataStreamRel if s.producesUpdates =>
+          isAppendOnly = false
+        case _ =>
+          super.visit(node, ordinal, parent)
+      }
+    }
+  }
+
+  /** Identifies unique key fields in the output of a RelNode. */
+  private class UniqueKeyExtractor extends RelVisitor {
+
+    var keys: Option[Array[String]] = None
+
+    override def visit(node: RelNode, ordinal: Int, parent: RelNode): Unit = {
+      node match {
+        case c: DataStreamCalc =>
+          super.visit(node, ordinal, parent)
+          // check if input has keys
+          if (keys.isDefined) {
+            // track keys forward
+            val inNames = c.getInput.getRowType.getFieldNames
+            val inOutNames = c.getProgram.getNamedProjects.asScala
+              .map(p => {
+                c.getProgram.expandLocalRef(p.left) match {
+                    // output field is forwarded input field
+                  case i: RexInputRef => (i.getIndex, p.right)
+                    // output field is renamed input field
+                  case a: RexCall if a.getKind.equals(SqlKind.AS) =>
+                    a.getOperands.get(0) match {
+                      case ref: RexInputRef =>
+                        (ref.getIndex, p.right)
+                      case _ =>
+                        (-1, p.right)
+                    }
+                    // output field is not forwarded from input
+                  case _: RexNode => (-1, p.right)
+                }
+              })
+              // filter all non-forwarded fields
+              .filter(_._1 >= 0)
+              // resolve names of input fields
+              .map(io => (inNames.get(io._1), io._2))
+
+            // filter by input keys
+            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
+            // check if all keys have been preserved
+            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
+              // all key have been preserved (but possibly renamed)
+              keys = Some(outKeys.toArray)
+            } else {
+              // some (or all) keys have been removed. Keys are no longer unique and removed
+              keys = None
+            }
+          }
+        case _: DataStreamOverAggregate =>
+          super.visit(node, ordinal, parent)
+          // keys are always forwarded by Over aggregate
+        case a: DataStreamGroupAggregate =>
+          // get grouping keys
+          val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
+          keys = Some(groupKeys.toArray)
+        case w: DataStreamGroupWindowAggregate =>
+          // get grouping keys
+          val groupKeys =
+            w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
+          // get window start and end time
+          val windowStartEnd = w.getWindowProperties.map(_.name)
+          // we have only a unique key if at least one window property is selected
+          if (windowStartEnd.nonEmpty) {
+            keys = Some(groupKeys ++ windowStartEnd)
+          }
+        case _: DataStreamRel =>
+          // anything else does not forward keys or might duplicate key, so we can stop
+          keys = None
+      }
+    }
+
+  }
+
 }
+

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
index d296978..6448657 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
@@ -37,11 +37,6 @@ class TableConfig {
   private var nullCheck: Boolean = true
 
   /**
-    * Defines whether sink table requires that update and delete changes are sent with retraction
-    */
-  private var needsUpdatesAsRetractionForSink: Boolean = false
-
-  /**
     * Defines the configuration of Calcite for Table API and SQL queries.
     */
   private var calciteConfig = CalciteConfig.DEFAULT
@@ -72,18 +67,6 @@ class TableConfig {
   }
 
   /**
-    * Returns the need retraction property for table sink.
-    */
-  def getNeedsUpdatesAsRetractionForSink = needsUpdatesAsRetractionForSink
-
-  /**
-    * Set the need retraction property for table sink.
-    */
-  def setNeedsUpdatesAsRetractionForSink(needsUpdatesAsRetraction: Boolean ): Unit = {
-    this.needsUpdatesAsRetractionForSink = needsUpdatesAsRetraction
-  }
-
-  /**
     * Returns the current configuration of Calcite for Table API and SQL queries.
     */
   def getCalciteConfig: CalciteConfig = calciteConfig

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 5b752ab..bb0de3e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -51,15 +51,14 @@ import org.apache.flink.table.catalog.{ExternalCatalog, ExternalCatalogSchema}
 import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer, GeneratedFunction}
 import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
-import org.apache.flink.table.functions.{ScalarFunction, TableFunction, AggregateFunction}
+import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.expressions._
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions}
 import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
 import org.apache.flink.table.plan.cost.DataSetCostFactory
 import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode}
 import org.apache.flink.table.plan.rules.FlinkRuleSets
-import org.apache.flink.table.plan.schema.RelTable
-import org.apache.flink.table.runtime.types.CRowTypeInfo
+import org.apache.flink.table.plan.schema.{RelTable, RowSchema}
 import org.apache.flink.table.sinks.TableSink
 import org.apache.flink.table.sources.{DefinedFieldNames, TableSource}
 import org.apache.flink.table.validate.FunctionCatalog
@@ -620,7 +619,7 @@ abstract class TableEnvironment(val config: TableConfig) {
         throw new TableException(
           "An input of GenericTypeInfo<Row> cannot be converted to Table. " +
             "Please specify the type of the input with a RowTypeInfo.")
-      case a: AtomicType[A] =>
+      case a: AtomicType[_] =>
         exprs.zipWithIndex flatMap {
           case (UnresolvedFieldReference(name), idx) =>
             if (idx > 0) {
@@ -691,53 +690,32 @@ abstract class TableEnvironment(val config: TableConfig) {
     (fieldNames.toArray, fieldIndexes.toArray)
   }
 
-  /**
-    * Creates a final converter that maps the internal row type to external type.
-    *
-    * @param physicalTypeInfo the input of the sink
-    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
-    * @param requestedTypeInfo the output type of the sink
-    * @param functionName name of the map function. Must not be unique but has to be a
-    *                     valid Java class identifier.
-    */
-  protected def getConversionMapper[IN, OUT](
-      physicalTypeInfo: TypeInformation[IN],
-      logicalRowType: RelDataType,
-      requestedTypeInfo: TypeInformation[OUT],
-      functionName: String):
-    Option[MapFunction[IN, OUT]]
-
   protected def generateRowConverterFunction[OUT](
       inputTypeInfo: TypeInformation[Row],
-      logicalRowType: RelDataType,
+      schema: RowSchema,
       requestedTypeInfo: TypeInformation[OUT],
       functionName: String):
     GeneratedFunction[MapFunction[Row, OUT], OUT] = {
 
     // validate that at least the field types of physical and logical type match
     // we do that here to make sure that plan translation was correct
-    val logicalRowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(logicalRowType)
-    if (logicalRowTypeInfo != inputTypeInfo) {
+    if (schema.physicalTypeInfo != inputTypeInfo) {
       throw TableException("The field types of physical and logical row types do not match." +
         "This is a bug and should not happen. Please file an issue.")
     }
 
-    // convert to type information
-    val logicalFieldTypes = logicalRowType.getFieldList.asScala
-      .map(t => FlinkTypeFactory.toTypeInfo(t.getType))
-
-    // field names
-    val logicalFieldNames = logicalRowType.getFieldNames.asScala
+    val fieldTypes = schema.physicalFieldTypeInfo
+    val fieldNames = schema.physicalFieldNames
 
     // validate requested type
-    if (requestedTypeInfo.getArity != logicalFieldTypes.length) {
+    if (requestedTypeInfo.getArity != fieldTypes.length) {
       throw new TableException("Arity of result does not match requested type.")
     }
 
     requestedTypeInfo match {
       // POJO type requested
       case pt: PojoTypeInfo[_] =>
-        logicalFieldNames.zip(logicalFieldTypes) foreach {
+        fieldNames.zip(fieldTypes) foreach {
           case (fName, fType) =>
             val pojoIdx = pt.getFieldIndex(fName)
             if (pojoIdx < 0) {
@@ -752,7 +730,7 @@ abstract class TableEnvironment(val config: TableConfig) {
 
       // Tuple/Case class/Row type requested
       case tt: TupleTypeInfoBase[_] =>
-        logicalFieldTypes.zipWithIndex foreach {
+        fieldTypes.zipWithIndex foreach {
           case (fieldTypeInfo, i) =>
             val requestedTypeInfo = tt.getTypeAt(i)
             if (fieldTypeInfo != requestedTypeInfo) {
@@ -763,11 +741,11 @@ abstract class TableEnvironment(val config: TableConfig) {
 
       // Atomic type requested
       case at: AtomicType[_] =>
-        if (logicalFieldTypes.size != 1) {
+        if (fieldTypes.size != 1) {
           throw new TableException(s"Requested result type is an atomic type but " +
             s"result has more or less than a single field.")
         }
-        val fieldTypeInfo = logicalFieldTypes.head
+        val fieldTypeInfo = fieldTypes.head
         if (fieldTypeInfo != at) {
           throw new TableException(s"Result field does not match requested type. " +
             s"Requested: $at; Actual: $fieldTypeInfo")
@@ -787,7 +765,7 @@ abstract class TableEnvironment(val config: TableConfig) {
 
     val conversion = generator.generateConverterResultExpression(
       requestedTypeInfo,
-      logicalFieldNames)
+      fieldNames)
 
     val body =
       s"""

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
index a649584..a70bcca 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
@@ -18,12 +18,14 @@
 package org.apache.flink.table.api.java
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
 import org.apache.flink.table.api._
 import org.apache.flink.table.functions.{AggregateFunction, TableFunction}
 import org.apache.flink.table.expressions.ExpressionParser
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import _root_.java.lang.{Boolean => JBool}
 
 /**
   * The [[TableEnvironment]] for a Java [[StreamExecutionEnvironment]].
@@ -132,7 +134,10 @@ class StreamTableEnvironment(
   }
 
   /**
-    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
+    * Converts the given [[Table]] into an append [[DataStream]] of a specified type.
+    *
+    * The [[Table]] must only have insert (append) changes. If the [[Table]] is also modified
+    * by update or delete changes, the conversion will fail.
     *
     * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows:
     * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
@@ -145,11 +150,16 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = {
-    translate[T](table)(TypeExtractor.createTypeInfo(clazz))
+    val typeInfo = TypeExtractor.createTypeInfo(clazz)
+    TableEnvironment.validateType(typeInfo)
+    translate[T](table, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
   }
 
   /**
-    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
+    * Converts the given [[Table]] into an append [[DataStream]] of a specified type.
+    *
+    * The [[Table]] must only have insert (append) changes. If the [[Table]] is also modified
+    * by update or delete changes, the conversion will fail.
     *
     * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows:
     * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
@@ -162,7 +172,68 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = {
-    translate[T](table)(typeInfo)
+    TableEnvironment.validateType(typeInfo)
+    translate[T](table, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataStream]] of add and retract messages.
+    * The message will be encoded as [[JTuple2]]. The first field is a [[JBool]] flag,
+    * the second field holds the record of the specified type [[T]].
+    *
+    * A true [[JBool]] flag indicates an add message, a false flag indicates a retract message.
+    *
+    * The fields of the [[Table]] are mapped to the requested type as follows:
+    * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
+    * types: Fields are mapped by position, field types must match.
+    * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match.
+    *
+    * @param table The [[Table]] to convert.
+    * @param clazz The class of the requested record type.
+    * @tparam T The type of the requested record type.
+    * @return The converted [[DataStream]].
+    */
+  def toRetractStream[T](table: Table, clazz: Class[T]):
+    DataStream[JTuple2[JBool, T]] = {
+
+    val typeInfo = TypeExtractor.createTypeInfo(clazz)
+    TableEnvironment.validateType(typeInfo)
+    val resultType = new TupleTypeInfo[JTuple2[JBool, T]](Types.BOOLEAN, typeInfo)
+    translate[JTuple2[JBool, T]](
+      table,
+      updatesAsRetraction = true,
+      withChangeFlag = true)(resultType)
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataStream]] of add and retract messages.
+    * The message will be encoded as [[JTuple2]]. The first field is a [[JBool]] flag,
+    * the second field holds the record of the specified type [[T]].
+    *
+    * A true [[JBool]] flag indicates an add message, a false flag indicates a retract message.
+    *
+    * The fields of the [[Table]] are mapped to the requested type as follows:
+    * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
+    * types: Fields are mapped by position, field types must match.
+    * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match.
+    *
+    * @param table The [[Table]] to convert.
+    * @param typeInfo The [[TypeInformation]] of the requested record type.
+    * @tparam T The type of the requested record type.
+    * @return The converted [[DataStream]].
+    */
+  def toRetractStream[T](table: Table, typeInfo: TypeInformation[T]):
+    DataStream[JTuple2[JBool, T]] = {
+
+    TableEnvironment.validateType(typeInfo)
+    val resultTypeInfo = new TupleTypeInfo[JTuple2[JBool, T]](
+      Types.BOOLEAN,
+      typeInfo
+    )
+    translate[JTuple2[JBool, T]](
+      table,
+      updatesAsRetraction = true,
+      withChangeFlag = true)(resultTypeInfo)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
index 0552d7c..e5ad6c2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
@@ -17,10 +17,11 @@
  */
 package org.apache.flink.table.api.scala
 
+import org.apache.flink.api.scala._
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.table.api.{TableEnvironment, Table, TableConfig}
-import org.apache.flink.table.functions.{AggregateFunction, TableFunction}
+import org.apache.flink.table.api.{Table, TableConfig, TableEnvironment}
 import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.functions.{AggregateFunction, TableFunction}
 import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
 import org.apache.flink.streaming.api.scala.asScalaStream
 
@@ -127,11 +128,14 @@ class StreamTableEnvironment(
   }
 
   /**
-    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
+    * Converts the given [[Table]] into an append [[DataStream]] of a specified type.
+    *
+    * The [[Table]] must only have insert (append) changes. If the [[Table]] is also modified
+    * by update or delete changes, the conversion will fail.
     *
     * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows:
-    * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
-    * types: Fields are mapped by position, field types must match.
+    * - [[org.apache.flink.types.Row]] and Scala Tuple types: Fields are mapped by position, field
+    * types must match.
     * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match.
     *
     * @param table The [[Table]] to convert.
@@ -139,7 +143,24 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T: TypeInformation](table: Table): DataStream[T] = {
-    asScalaStream(translate(table))
+    val returnType = createTypeInformation[T]
+    asScalaStream(translate(table, updatesAsRetraction = false, withChangeFlag = false)(returnType))
+  }
+
+/**
+  * Converts the given [[Table]] into a [[DataStream]] of add and retract messages.
+  * The message will be encoded as [[Tuple2]]. The first field is a [[Boolean]] flag,
+  * the second field holds the record of the specified type [[T]].
+  *
+  * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message.
+  *
+  * @param table The [[Table]] to convert.
+  * @tparam T The type of the requested data type.
+  * @return The converted [[DataStream]].
+  */
+  def toRetractStream[T: TypeInformation](table: Table): DataStream[(Boolean, T)] = {
+    val returnType = createTypeInformation[(Boolean, T)]
+    asScalaStream(translate(table, updatesAsRetraction = true, withChangeFlag = true)(returnType))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
index 2a0d571..5efff62 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
@@ -57,5 +57,24 @@ class TableConversions(table: Table) {
     }
   }
 
+  /** Converts the [[Table]] to a [[DataStream]] of add and retract messages.
+    * The message will be encoded as [[Tuple2]]. The first field is a [[Boolean]] flag,
+    * the second field holds the record of the specified type [[T]].
+    *
+    * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message.
+    *
+    */
+  def toRetractStream[T: TypeInformation]: DataStream[(Boolean, T)] = {
+
+    table.tableEnv match {
+      case tEnv: ScalaStreamTableEnv =>
+        tEnv.toRetractStream(table)
+      case _ =>
+        throw new TableException(
+          "Only tables that originate from Scala DataStreams " +
+            "can be converted to Scala DataStreams.")
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
index dd8265b..310a75f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
@@ -755,7 +755,9 @@ class Table(
     *
     * A batch [[Table]] can only be written to a
     * [[org.apache.flink.table.sinks.BatchTableSink]], a streaming [[Table]] requires a
-    * [[org.apache.flink.table.sinks.StreamTableSink]].
+    * [[org.apache.flink.table.sinks.AppendStreamTableSink]], a
+    * [[org.apache.flink.table.sinks.RetractStreamTableSink]], or an
+    * [[org.apache.flink.table.sinks.UpsertStreamTableSink]].
     *
     * @param sink The [[TableSink]] to which the [[Table]] is written.
     * @tparam T The data type that the [[TableSink]] expects.

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
index 59f723ac..ce0f966 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
@@ -100,11 +100,17 @@ class DataStreamCalc(
       calcProgram,
       config)
 
+    val inputParallelism = inputDataStream.getParallelism
+
     val mapFunc = new CRowFlatMapRunner(
       genFunction.name,
       genFunction.code,
       CRowTypeInfo(schema.physicalTypeInfo))
 
-    inputDataStream.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
+    inputDataStream
+      .flatMap(mapFunc)
+      .name(calcOpName(calcProgram, getExpressionString))
+      // keep parallelism to ensure order of accumulate and retract messages
+      .setParallelism(inputParallelism)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
index 056cda9..18f1fc8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
@@ -67,6 +67,8 @@ class DataStreamGroupAggregate(
 
   override def consumesRetractions = true
 
+  def getGroupings: Array[Int] = groupings
+
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamGroupAggregate(
       cluster,

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
index f61828b..1be1896 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
@@ -59,6 +59,10 @@ class DataStreamGroupWindowAggregate(
 
   override def consumesRetractions = true
 
+  def getGroupings: Array[Int] = grouping
+
+  def getWindowProperties: Seq[NamedWindowProperty] = namedProperties
+
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamGroupWindowAggregate(
       window,

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
index c3b43ba..173b7d3 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
@@ -82,19 +82,38 @@ object AccModeTrait {
 }
 
 /**
-  * The AccMode indicates which kinds of messages a [[org.apache.calcite.rel.RelNode]] might
-  * produce.
-  * In [[AccMode.Acc]] the node only emit accumulate messages.
-  * In [[AccMode.AccRetract]], the node produces accumulate messages for insert changes,
-  * retraction messages for delete changes, and accumulate and retraction messages
-  * for update changes.
+  * The [[AccMode]] determines how insert, update, and delete changes of tables are encoded
+  * by the messeages that an operator emits.
   */
 object AccMode extends Enumeration {
   type AccMode = Value
 
-  val Acc        = Value // Operator produces only accumulate (insert) messages
-  val AccRetract = Value // Operator produces accumulate (insert, update) and
-                         //   retraction (delete, update) messages
+  /**
+    * An operator in [[Acc]] mode emits change messages as
+    * [[org.apache.flink.table.runtime.types.CRow]] which encode a pair of (Boolean, Row).
+    *
+    * An operator in [[Acc]] mode may only produce update and delete messages, if the table has
+    * a unique key and all key attributes are contained in the Row.
+    *
+    * Changes are encoded as follows:
+    * - insert: (true, NewRow)
+    * - update: (true, NewRow) // the Row includes the full unique key to identify the row to update
+    * - delete: (false, OldRow) // the Row includes the full unique key to idenify the row to delete
+    *
+    */
+  val Acc = Value
+
+  /**
+    * * An operator in [[AccRetract]] mode emits change messages as
+    * [[org.apache.flink.table.runtime.types.CRow]] which encode a pair of (Boolean, Row).
+    *
+    * Changes are encoded as follows:
+    * - insert: (true, NewRow)
+    * - update: (false, OldRow), (true, NewRow) // updates are encoded in two messages!
+    * - delete: (false, OldRow)
+    *
+    */
+  val AccRetract = Value
 }
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
index 97c0dbb..f0b725d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
@@ -102,17 +102,17 @@ object DataStreamRetractionRules {
       val rel = call.rel(0).asInstanceOf[DataStreamRel]
       val traits = rel.getTraitSet
 
-      val traitsWithUpdateAsRetrac =
+      val traitsWithUpdateAsRetraction =
         if (null == traits.getTrait(UpdateAsRetractionTraitDef.INSTANCE)) {
         traits.plus(UpdateAsRetractionTrait.DEFAULT)
       } else {
         traits
       }
       val traitsWithAccMode =
-        if (null == traitsWithUpdateAsRetrac.getTrait(AccModeTraitDef.INSTANCE)) {
-          traitsWithUpdateAsRetrac.plus(AccModeTrait.DEFAULT)
+        if (null == traitsWithUpdateAsRetraction.getTrait(AccModeTraitDef.INSTANCE)) {
+          traitsWithUpdateAsRetraction.plus(AccModeTrait.DEFAULT)
       } else {
-        traitsWithUpdateAsRetrac
+        traitsWithUpdateAsRetraction
       }
 
       if (traits != traitsWithAccMode) {
@@ -122,8 +122,8 @@ object DataStreamRetractionRules {
   }
 
   /**
-    * Rule that annotates all [[DataStreamRel]] nodes that need to sent out update and delete
-    * changes as retraction messages.
+    * Rule that annotates all [[DataStreamRel]] nodes that need to sent out update changes with
+    * retraction messages.
     */
   class SetUpdatesAsRetractionRule extends RelOptRule(
     operand(
@@ -131,7 +131,7 @@ object DataStreamRetractionRules {
     "SetUpdatesAsRetractionRule") {
 
     /**
-      * Checks if a [[RelNode]] requires that update and delete changes are sent with retraction
+      * Checks if a [[RelNode]] requires that update changes are sent with retraction
       * messages.
       */
     def needsUpdatesAsRetraction(node: RelNode): Boolean = {
@@ -142,7 +142,7 @@ object DataStreamRetractionRules {
     }
 
     /**
-      * Annotates a [[RelNode]] to send out update and delete changes as retraction messages.
+      * Annotates a [[RelNode]] to send out update changes with retraction messages.
       */
     def setUpdatesAsRetraction(relNode: RelNode): RelNode = {
       val traitSet = relNode.getTraitSet

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
index 66e51b1..ff3821a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
@@ -37,7 +37,7 @@ class CRowCorrelateFlatMapRunner(
     flatMapCode: String,
     collectorName: String,
     collectorCode: String,
-    @transient returnType: TypeInformation[CRow])
+    @transient var returnType: TypeInformation[CRow])
   extends RichFlatMapFunction[CRow, CRow]
   with ResultTypeQueryable[CRow]
   with Compiler[Any] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
index 9a4650b..9701cb9 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
@@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory
 class CRowFlatMapRunner(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[CRow])
+    @transient var returnType: TypeInformation[CRow])
   extends RichFlatMapFunction[CRow, CRow]
   with ResultTypeQueryable[CRow]
   with Compiler[FlatMapFunction[Row, Row]] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
index 8e95c93..109c6e1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
@@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory
 class CRowInputMapRunner[OUT](
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT])
+    @transient var returnType: TypeInformation[OUT])
   extends RichMapFunction[CRow, OUT]
   with ResultTypeQueryable[OUT]
   with Compiler[MapFunction[Row, OUT]] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
index 54bbf7e..7c96437 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
@@ -18,6 +18,8 @@
 
 package org.apache.flink.table.runtime
 
+import java.lang.{Boolean => JBool}
+
 import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
@@ -28,36 +30,63 @@ import org.apache.flink.types.Row
 import org.slf4j.LoggerFactory
 import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
 
-
 /**
-  * Convert [[CRow]] to a [[Tuple2]]
+  * Convert [[CRow]] to a [[JTuple2]]
   */
-class CRowInputTupleOutputMapRunner[OUT](
+class CRowInputJavaTupleOutputMapRunner(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[JTuple2[Boolean, OUT]])
-  extends RichMapFunction[CRow, JTuple2[Boolean, OUT]]
-          with ResultTypeQueryable[JTuple2[Boolean, OUT]]
-          with Compiler[MapFunction[Row, OUT]] {
+    @transient var returnType: TypeInformation[JTuple2[JBool, Any]])
+  extends RichMapFunction[CRow, Any]
+          with ResultTypeQueryable[JTuple2[JBool, Any]]
+          with Compiler[MapFunction[Row, Any]] {
 
   val LOG = LoggerFactory.getLogger(this.getClass)
 
-  private var function: MapFunction[Row, OUT] = _
-  private var tupleWrapper: JTuple2[Boolean, OUT] = _
+  private var function: MapFunction[Row, Any] = _
+  private var tupleWrapper: JTuple2[JBool, Any] = _
 
   override def open(parameters: Configuration): Unit = {
     LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating MapFunction.")
     function = clazz.newInstance()
-    tupleWrapper = new JTuple2[Boolean, OUT]()
+    tupleWrapper = new JTuple2[JBool, Any]()
   }
 
-  override def map(in: CRow): JTuple2[Boolean, OUT] = {
+  override def map(in: CRow): JTuple2[JBool, Any] = {
     tupleWrapper.f0 = in.change
     tupleWrapper.f1 = function.map(in.row)
     tupleWrapper
   }
 
-  override def getProducedType: TypeInformation[JTuple2[Boolean, OUT]] = returnType
+  override def getProducedType: TypeInformation[JTuple2[JBool, Any]] = returnType
+}
+
+/**
+  * Convert [[CRow]] to a [[Tuple2]]
+  */
+class CRowInputScalaTupleOutputMapRunner(
+  name: String,
+  code: String,
+  @transient var returnType: TypeInformation[(Boolean, Any)])
+  extends RichMapFunction[CRow, (Boolean, Any)]
+    with ResultTypeQueryable[(Boolean, Any)]
+    with Compiler[MapFunction[Row, Any]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: MapFunction[Row, Any] = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating MapFunction.")
+    function = clazz.newInstance()
+  }
+
+  override def map(in: CRow): (Boolean, Any) =
+    (in.change, function.map(in.row))
+
+  override def getProducedType: TypeInformation[(Boolean, Any)] = returnType
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
index 966dea9..cb8f695 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
@@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory
 class CRowOutputMapRunner(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[CRow])
+    @transient var returnType: TypeInformation[CRow])
   extends RichMapFunction[Any, CRow]
   with ResultTypeQueryable[CRow]
   with Compiler[MapFunction[Any, Row]] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala
index a0415e1..478b6b6 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala
@@ -32,7 +32,7 @@ class CorrelateFlatMapRunner[IN, OUT](
     flatMapCode: String,
     collectorName: String,
     collectorCode: String,
-    @transient returnType: TypeInformation[OUT])
+    @transient var returnType: TypeInformation[OUT])
   extends RichFlatMapFunction[IN, OUT]
   with ResultTypeQueryable[OUT]
   with Compiler[Any] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
index 715848d..67acc0b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
@@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory
 class FlatJoinRunner[IN1, IN2, OUT](
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT])
+    @transient var returnType: TypeInformation[OUT])
   extends RichFlatJoinFunction[IN1, IN2, OUT]
   with ResultTypeQueryable[OUT]
   with Compiler[FlatJoinFunction[IN1, IN2, OUT]] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
index 2e37baf..938da59 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory
 class FlatMapRunner(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[Row])
+    @transient var returnType: TypeInformation[Row])
   extends RichFlatMapFunction[Row, Row]
   with ResultTypeQueryable[Row]
   with Compiler[FlatMapFunction[Row, Row]] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
index 644e855..5f3dbb4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
@@ -24,7 +24,7 @@ import org.apache.flink.util.Collector
 class MapJoinLeftRunner[IN1, IN2, OUT](
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT],
+    returnType: TypeInformation[OUT],
     broadcastSetName: String)
   extends MapSideJoinRunner[IN1, IN2, IN2, IN1, OUT](name, code, returnType, broadcastSetName) {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
index eee38d1..e2d9331 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
@@ -24,7 +24,7 @@ import org.apache.flink.util.Collector
 class MapJoinRightRunner[IN1, IN2, OUT](
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT],
+    returnType: TypeInformation[OUT],
     broadcastSetName: String)
   extends MapSideJoinRunner[IN1, IN2, IN1, IN2, OUT](name, code, returnType, broadcastSetName) {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
index 32562c7..14eeecf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
@@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory
 class MapRunner[IN, OUT](
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT])
+    @transient var returnType: TypeInformation[OUT])
   extends RichMapFunction[IN, OUT]
   with ResultTypeQueryable[OUT]
   with Compiler[MapFunction[IN, OUT]] {

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
index 090e184..00b7b8e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
@@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory
 abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT](
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT],
+    @transient var returnType: TypeInformation[OUT],
     broadcastSetName: String)
   extends RichFlatMapFunction[MULTI_IN, OUT]
     with ResultTypeQueryable[OUT]

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala
index 22a2682..9bcac30 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala
@@ -41,7 +41,7 @@ class DataSetSessionWindowAggregatePreProcessor(
     genAggregations: GeneratedAggregationsFunction,
     keysAndAggregatesArity: Int,
     gap: Long,
-    @transient intermediateRowType: TypeInformation[Row])
+    @transient var intermediateRowType: TypeInformation[Row])
   extends AbstractRichFunction
   with MapPartitionFunction[Row,Row]
   with GroupCombineFunction[Row,Row]

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
index 745f24d..6ee37e6 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
@@ -17,6 +17,8 @@
  */
 package org.apache.flink.table.runtime.aggregate
 
+import java.lang.{Long => JLong}
+
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.types.Row
@@ -24,6 +26,7 @@ import org.apache.flink.util.Collector
 import org.apache.flink.api.common.state.ValueStateDescriptor
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.table.api.Types
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.slf4j.LoggerFactory
 import org.apache.flink.table.runtime.types.CRow
@@ -47,7 +50,10 @@ class GroupAggProcessFunction(
   private var newRow: CRow = _
   private var prevRow: CRow = _
   private var firstRow: Boolean = _
+  // stores the accumulators
   private var state: ValueState[Row] = _
+  // counts the number of added and retracted input records
+  private var cntState: ValueState[JLong] = _
 
   override def open(config: Configuration) {
     LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " +
@@ -65,6 +71,9 @@ class GroupAggProcessFunction(
     val stateDescriptor: ValueStateDescriptor[Row] =
       new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType)
     state = getRuntimeContext.getState(stateDescriptor)
+    val inputCntDescriptor: ValueStateDescriptor[JLong] =
+      new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG)
+    cntState = getRuntimeContext.getState(inputCntDescriptor)
   }
 
   override def processElement(
@@ -74,11 +83,14 @@ class GroupAggProcessFunction(
 
     val input = inputC.row
 
-    // get accumulators
+    // get accumulators and input counter
     var accumulators = state.value()
+    var inputCnt = cntState.value()
+
     if (null == accumulators) {
       firstRow = true
       accumulators = function.createAccumulators()
+      inputCnt = 0L
     } else {
       firstRow = false
     }
@@ -92,29 +104,44 @@ class GroupAggProcessFunction(
 
     // update aggregate result and set to the newRow
     if (inputC.change) {
+      inputCnt += 1
       // accumulate input
       function.accumulate(accumulators, input)
       function.setAggregationResults(accumulators, newRow.row)
     } else {
+      inputCnt -= 1
       // retract input
       function.retract(accumulators, input)
       function.setAggregationResults(accumulators, newRow.row)
     }
 
-    // update accumulators
-    state.update(accumulators)
-
-    // if previousRow is not null, do retraction process
-    if (generateRetraction && !firstRow) {
-      if (prevRow.row.equals(newRow.row)) {
-        // ignore same newRow
-        return
-      } else {
-        // retract previous row
-        out.collect(prevRow)
+    if (inputCnt != 0) {
+      // we aggregated at least one record for this key
+
+      // update the state
+      state.update(accumulators)
+      cntState.update(inputCnt)
+
+      // if this was not the first row and we have to emit retractions
+      if (generateRetraction && !firstRow) {
+        if (prevRow.row.equals(newRow.row)) {
+          // newRow is the same as before. Do not emit retraction and acc messages
+          return
+        } else {
+          // retract previous result
+          out.collect(prevRow)
+        }
       }
-    }
+      // emit the new result
+      out.collect(newRow)
 
-    out.collect(newRow)
+    } else {
+      // we retracted the last record for this key
+      // sent out a delete message
+      out.collect(prevRow)
+      // and clear all state
+      state.clear()
+      cntState.clear()
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
index ec73fa6..1cb3a6e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory
 class CRowValuesInputFormat(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[CRow])
+    @transient var returnType: TypeInformation[CRow])
   extends GenericInputFormat[CRow]
   with NonParallelInput
   with ResultTypeQueryable[CRow]

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
index d536b39..43ce605 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
@@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory
 class ValuesInputFormat(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[Row])
+    @transient var returnType: TypeInformation[Row])
   extends GenericInputFormat[Row]
   with NonParallelInput
   with ResultTypeQueryable[Row]

http://git-wip-us.apache.org/repos/asf/flink/blob/f37988c1/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
new file mode 100644
index 0000000..abdca17
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.table.sinks
+
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.Table
+
+/**
+  * Defines an external [[TableSink]] to emit streaming [[Table]] with only insert changes.
+  *
+  * If the [[Table]] is also modified by update or delete changes, a
+  * [[org.apache.flink.table.api.TableException]] will be thrown.
+  *
+  * @tparam T Type of [[DataStream]] that this [[TableSink]] expects and supports.
+  */
+trait AppendStreamTableSink[T] extends TableSink[T] {
+
+  /** Emits the DataStream. */
+  def emitDataStream(dataStream: DataStream[T]): Unit
+}


[11/15] flink git commit: [FLINK-6091] [table] Implement and turn on retraction for non-windowed aggregates.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
new file mode 100644
index 0000000..8e95c93
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala
@@ -0,0 +1,57 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.slf4j.LoggerFactory
+
+/**
+  * MapRunner with [[CRow]] input.
+  */
+class CRowInputMapRunner[OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT])
+  extends RichMapFunction[CRow, OUT]
+  with ResultTypeQueryable[OUT]
+  with Compiler[MapFunction[Row, OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: MapFunction[Row, OUT] = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating MapFunction.")
+    function = clazz.newInstance()
+  }
+
+  override def map(in: CRow): OUT = {
+    function.map(in.row)
+  }
+
+  override def getProducedType: TypeInformation[OUT] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
new file mode 100644
index 0000000..966dea9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala
@@ -0,0 +1,60 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.slf4j.LoggerFactory
+
+/**
+  * MapRunner with [[CRow]] output.
+  */
+class CRowOutputMapRunner(
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[CRow])
+  extends RichMapFunction[Any, CRow]
+  with ResultTypeQueryable[CRow]
+  with Compiler[MapFunction[Any, Row]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: MapFunction[Any, Row] = _
+  private var outCRow: CRow = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating MapFunction.")
+    function = clazz.newInstance()
+    outCRow = new CRow(null, true)
+  }
+
+  override def map(in: Any): CRow = {
+    outCRow.row = function.map(in)
+    outCRow
+  }
+
+  override def getProducedType: TypeInformation[CRow] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowWrappingCollector.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowWrappingCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowWrappingCollector.scala
new file mode 100644
index 0000000..b2b062e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowWrappingCollector.scala
@@ -0,0 +1,41 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * The collector is used to wrap a [[Row]] to a [[CRow]]
+  */
+class CRowWrappingCollector() extends Collector[Row] {
+
+  var out: Collector[CRow] = _
+  val outCRow: CRow = new CRow()
+
+  def setChange(change: Boolean): Unit = this.outCRow.change = change
+
+  override def collect(record: Row): Unit = {
+    outCRow.row = record
+    out.collect(outCRow)
+  }
+
+  override def close(): Unit = out.close()
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
index b446306..2e37baf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
@@ -24,20 +24,21 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
 import org.apache.flink.table.codegen.Compiler
 import org.apache.flink.configuration.Configuration
+import org.apache.flink.types.Row
 import org.apache.flink.util.Collector
 import org.slf4j.LoggerFactory
 
-class FlatMapRunner[IN, OUT](
+class FlatMapRunner(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT])
-  extends RichFlatMapFunction[IN, OUT]
-  with ResultTypeQueryable[OUT]
-  with Compiler[FlatMapFunction[IN, OUT]] {
+    @transient returnType: TypeInformation[Row])
+  extends RichFlatMapFunction[Row, Row]
+  with ResultTypeQueryable[Row]
+  with Compiler[FlatMapFunction[Row, Row]] {
 
   val LOG = LoggerFactory.getLogger(this.getClass)
 
-  private var function: FlatMapFunction[IN, OUT] = _
+  private var function: FlatMapFunction[Row, Row] = _
 
   override def open(parameters: Configuration): Unit = {
     LOG.debug(s"Compiling FlatMapFunction: $name \n\n Code:\n$code")
@@ -48,10 +49,10 @@ class FlatMapRunner[IN, OUT](
     FunctionUtils.openFunction(function, parameters)
   }
 
-  override def flatMap(in: IN, out: Collector[OUT]): Unit =
+  override def flatMap(in: Row, out: Collector[Row]): Unit =
     function.flatMap(in, out)
 
-  override def getProducedType: TypeInformation[OUT] = returnType
+  override def getProducedType: TypeInformation[Row] = returnType
 
   override def close(): Unit = {
     FunctionUtils.closeFunction(function)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala
index 377e0ff..dd9c015 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala
@@ -20,6 +20,7 @@ package org.apache.flink.table.runtime.aggregate
 
 import org.apache.flink.api.common.functions.AggregateFunction
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.types.Row
 import org.slf4j.LoggerFactory
 
@@ -30,28 +31,28 @@ import org.slf4j.LoggerFactory
   * @param genAggregations Generated aggregate helper function
   */
 class AggregateAggFunction(genAggregations: GeneratedAggregationsFunction)
-  extends AggregateFunction[Row, Row, Row] with Compiler[GeneratedAggregations] {
+  extends AggregateFunction[CRow, Row, Row] with Compiler[GeneratedAggregations] {
 
   val LOG = LoggerFactory.getLogger(this.getClass)
   private var function: GeneratedAggregations = _
 
   override def createAccumulator(): Row = {
     if (function == null) {
-      initFunction
+      initFunction()
     }
     function.createAccumulators()
   }
 
-  override def add(value: Row, accumulatorRow: Row): Unit = {
+  override def add(value: CRow, accumulatorRow: Row): Unit = {
     if (function == null) {
-      initFunction
+      initFunction()
     }
-    function.accumulate(accumulatorRow, value)
+    function.accumulate(accumulatorRow, value.row)
   }
 
   override def getResult(accumulatorRow: Row): Row = {
     if (function == null) {
-      initFunction
+      initFunction()
     }
     val output = function.createOutputRow()
     function.setAggregationResults(accumulatorRow, output)
@@ -60,7 +61,7 @@ class AggregateAggFunction(genAggregations: GeneratedAggregationsFunction)
 
   override def merge(aAccumulatorRow: Row, bAccumulatorRow: Row): Row = {
     if (function == null) {
-      initFunction
+      initFunction()
     }
     function.mergeAccumulatorsPair(aAccumulatorRow, bAccumulatorRow)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index 5e9efd0..768c9cb 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -44,6 +44,7 @@ import org.apache.flink.table.functions.utils.AggSqlFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{AggregateFunction => TableAggregateFunction}
 import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.apache.flink.table.typeutils.TypeCheckUtils._
 import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
 import org.apache.flink.types.Row
@@ -79,7 +80,7 @@ object AggregateUtil {
       isRowTimeType: Boolean,
       isPartitioned: Boolean,
       isRowsClause: Boolean)
-    : ProcessFunction[Row, Row] = {
+    : ProcessFunction[CRow, CRow] = {
 
     val (aggFields, aggregates) =
       transformToAggregateFunctions(
@@ -116,13 +117,13 @@ object AggregateUtil {
         new RowTimeUnboundedRowsOver(
           genFunction,
           aggregationStateType,
-          inputTypeInfo)
+          CRowTypeInfo(inputTypeInfo))
       } else {
         // RANGE unbounded over process function
         new RowTimeUnboundedRangeOver(
           genFunction,
           aggregationStateType,
-          inputTypeInfo)
+          CRowTypeInfo(inputTypeInfo))
       }
     } else {
       if (isPartitioned) {
@@ -153,13 +154,16 @@ object AggregateUtil {
       namedAggregates: Seq[CalcitePair[AggregateCall, String]],
       inputRowType: RelDataType,
       inputFieldTypes: Seq[TypeInformation[_]],
-      groupings: Array[Int]): ProcessFunction[Row, Row] = {
+      groupings: Array[Int],
+      generateRetraction: Boolean,
+      consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = {
 
     val (aggFields, aggregates) =
       transformToAggregateFunctions(
         namedAggregates.map(_.getKey),
         inputRowType,
-        needRetraction = false)
+        consumeRetraction)
+
     val aggMapping = aggregates.indices.map(_ + groupings.length).toArray
 
     val outputArity = groupings.length + aggregates.length
@@ -178,14 +182,16 @@ object AggregateUtil {
       None,
       None,
       outputArity,
-      needRetract = false,
+      consumeRetraction,
       needMerge = false,
       needReset = false
     )
 
     new GroupAggProcessFunction(
       genFunction,
-      aggregationStateType)
+      aggregationStateType,
+      generateRetraction)
+
   }
 
   /**
@@ -198,7 +204,7 @@ object AggregateUtil {
     * @param inputTypeInfo Physical type information of the row.
     * @param inputFieldTypeInfo Physical type information of the row's fields.
     * @param precedingOffset the preceding offset
-    * @param isRowsClause   It is a tag that indicates whether the OVER clause is ROWS clause
+    * @param isRowsClause    It is a tag that indicates whether the OVER clause is ROWS clause
     * @param isRowTimeType   It is a tag that indicates whether the time type is rowTimeType
     * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
     */
@@ -211,7 +217,7 @@ object AggregateUtil {
       precedingOffset: Long,
       isRowsClause: Boolean,
       isRowTimeType: Boolean)
-    : ProcessFunction[Row, Row] = {
+    : ProcessFunction[CRow, CRow] = {
 
     val needRetract = true
     val (aggFields, aggregates) =
@@ -221,6 +227,7 @@ object AggregateUtil {
         needRetract)
 
     val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
+    val inputRowType = CRowTypeInfo(inputTypeInfo)
 
     val forwardMapping = (0 until inputType.getFieldCount).toArray
     val aggMapping = aggregates.indices.map(x => x + inputType.getFieldCount).toArray
@@ -248,14 +255,14 @@ object AggregateUtil {
         new RowTimeBoundedRowsOver(
           genFunction,
           aggregationStateType,
-          inputTypeInfo,
+          inputRowType,
           precedingOffset
         )
       } else {
         new RowTimeBoundedRangeOver(
           genFunction,
           aggregationStateType,
-          inputTypeInfo,
+          inputRowType,
           precedingOffset
         )
       }
@@ -265,13 +272,13 @@ object AggregateUtil {
           genFunction,
           precedingOffset,
           aggregationStateType,
-          inputTypeInfo)
+          inputRowType)
       } else {
         new ProcTimeBoundedRangeOver(
           genFunction,
           precedingOffset,
           aggregationStateType,
-          inputTypeInfo)
+          inputRowType)
       }
     }
   }
@@ -932,7 +939,7 @@ object AggregateUtil {
       window: LogicalWindow,
       finalRowArity: Int,
       properties: Seq[NamedWindowProperty])
-    : AllWindowFunction[Row, Row, DataStreamWindow] = {
+    : AllWindowFunction[Row, CRow, DataStreamWindow] = {
 
     if (isTimeWindow(window)) {
       val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
@@ -940,7 +947,7 @@ object AggregateUtil {
         startPos,
         endPos,
         finalRowArity)
-        .asInstanceOf[AllWindowFunction[Row, Row, DataStreamWindow]]
+        .asInstanceOf[AllWindowFunction[Row, CRow, DataStreamWindow]]
     } else {
       new IncrementalAggregateAllWindowFunction(
         finalRowArity)
@@ -955,8 +962,8 @@ object AggregateUtil {
       numGroupingKeys: Int,
       numAggregates: Int,
       finalRowArity: Int,
-      properties: Seq[NamedWindowProperty])
-    : WindowFunction[Row, Row, Tuple, DataStreamWindow] = {
+      properties: Seq[NamedWindowProperty]):
+    WindowFunction[Row, CRow, Tuple, DataStreamWindow] = {
 
     if (isTimeWindow(window)) {
       val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
@@ -966,7 +973,7 @@ object AggregateUtil {
         startPos,
         endPos,
         finalRowArity)
-        .asInstanceOf[WindowFunction[Row, Row, Tuple, DataStreamWindow]]
+        .asInstanceOf[WindowFunction[Row, CRow, Tuple, DataStreamWindow]]
     } else {
       new IncrementalAggregateWindowFunction(
         numGroupingKeys,
@@ -981,8 +988,9 @@ object AggregateUtil {
       inputType: RelDataType,
       inputFieldTypeInfo: Seq[TypeInformation[_]],
       outputType: RelDataType,
+      groupingKeys: Array[Int],
       needMerge: Boolean)
-    : (DataStreamAggFunction[Row, Row, Row], RowTypeInfo, RowTypeInfo) = {
+    : (DataStreamAggFunction[CRow, Row, Row], RowTypeInfo, RowTypeInfo) = {
 
     val needRetract = false
     val (aggFields, aggregates) =
@@ -1002,7 +1010,7 @@ object AggregateUtil {
       aggFields,
       aggMapping,
       partialResults = false,
-      Array(), // no fields are forwarded
+      groupingKeys,
       None,
       None,
       outputArity,

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala
index 95699a2..fabf200 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala
@@ -56,7 +56,7 @@ class DataSetSessionWindowAggReduceGroupFunction(
   extends RichGroupReduceFunction[Row, Row]
     with Compiler[GeneratedAggregations] {
 
-  private var collector: TimeWindowPropertyCollector = _
+  private var collector: RowTimeWindowPropertyCollector = _
   private val intermediateRowWindowStartPos = keysAndAggregatesArity
   private val intermediateRowWindowEndPos = keysAndAggregatesArity + 1
 
@@ -78,7 +78,7 @@ class DataSetSessionWindowAggReduceGroupFunction(
 
     output = function.createOutputRow()
     accumulators = function.createAccumulators()
-    collector = new TimeWindowPropertyCollector(finalRowWindowStartPos, finalRowWindowEndPos)
+    collector = new RowTimeWindowPropertyCollector(finalRowWindowStartPos, finalRowWindowEndPos)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala
index a221c53..56ed08a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala
@@ -47,7 +47,7 @@ class DataSetSlideWindowAggReduceGroupFunction(
   extends RichGroupReduceFunction[Row, Row]
     with Compiler[GeneratedAggregations] {
 
-  private var collector: TimeWindowPropertyCollector = _
+  private var collector: RowTimeWindowPropertyCollector = _
   protected val windowStartPos: Int = keysAndAggregatesArity
 
   private var output: Row = _
@@ -68,7 +68,7 @@ class DataSetSlideWindowAggReduceGroupFunction(
 
     output = function.createOutputRow()
     accumulators = function.createAccumulators()
-    collector = new TimeWindowPropertyCollector(finalRowWindowStartPos, finalRowWindowEndPos)
+    collector = new RowTimeWindowPropertyCollector(finalRowWindowStartPos, finalRowWindowEndPos)
   }
 
   override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = {

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala
index f4a1fc5..8af2c2e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala
@@ -46,7 +46,7 @@ class DataSetTumbleTimeWindowAggReduceGroupFunction(
   extends RichGroupReduceFunction[Row, Row]
     with Compiler[GeneratedAggregations] {
 
-  private var collector: TimeWindowPropertyCollector = _
+  private var collector: RowTimeWindowPropertyCollector = _
   protected var aggregateBuffer: Row = new Row(keysAndAggregatesArity + 1)
 
   private var output: Row = _
@@ -67,7 +67,7 @@ class DataSetTumbleTimeWindowAggReduceGroupFunction(
 
     output = function.createOutputRow()
     accumulators = function.createAccumulators()
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    collector = new RowTimeWindowPropertyCollector(windowStartPos, windowEndPos)
   }
 
   override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = {

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
index 81c900c..745f24d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.api.common.state.ValueState
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.slf4j.LoggerFactory
+import org.apache.flink.table.runtime.types.CRow
 
 /**
   * Aggregate Function used for the groupby (without window) aggregate
@@ -35,14 +36,17 @@ import org.slf4j.LoggerFactory
   */
 class GroupAggProcessFunction(
     private val genAggregations: GeneratedAggregationsFunction,
-    private val aggregationStateType: RowTypeInfo)
-  extends ProcessFunction[Row, Row]
+    private val aggregationStateType: RowTypeInfo,
+    private val generateRetraction: Boolean)
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
 
   val LOG = LoggerFactory.getLogger(this.getClass)
   private var function: GeneratedAggregations = _
 
-  private var output: Row = _
+  private var newRow: CRow = _
+  private var prevRow: CRow = _
+  private var firstRow: Boolean = _
   private var state: ValueState[Row] = _
 
   override def open(config: Configuration) {
@@ -54,7 +58,9 @@ class GroupAggProcessFunction(
       genAggregations.code)
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
-    output = function.createOutputRow()
+
+    newRow = new CRow(function.createOutputRow(), true)
+    prevRow = new CRow(function.createOutputRow(), false)
 
     val stateDescriptor: ValueStateDescriptor[Row] =
       new ValueStateDescriptor[Row]("GroupAggregateState", aggregationStateType)
@@ -62,29 +68,53 @@ class GroupAggProcessFunction(
   }
 
   override def processElement(
-      input: Row,
-      ctx: ProcessFunction[Row, Row]#Context,
-      out: Collector[Row]): Unit = {
+      inputC: CRow,
+      ctx: ProcessFunction[CRow, CRow]#Context,
+      out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
     // get accumulators
     var accumulators = state.value()
     if (null == accumulators) {
+      firstRow = true
       accumulators = function.createAccumulators()
+    } else {
+      firstRow = false
     }
 
     // Set group keys value to the final output
-    function.setForwardedFields(input, output)
+    function.setForwardedFields(input, newRow.row)
+    function.setForwardedFields(input, prevRow.row)
 
-    // accumulate new input row
-    function.accumulate(accumulators, input)
+    // Set previous aggregate result to the prevRow
+    function.setAggregationResults(accumulators, prevRow.row)
 
-    // set aggregation results to output
-    function.setAggregationResults(accumulators, output)
+    // update aggregate result and set to the newRow
+    if (inputC.change) {
+      // accumulate input
+      function.accumulate(accumulators, input)
+      function.setAggregationResults(accumulators, newRow.row)
+    } else {
+      // retract input
+      function.retract(accumulators, input)
+      function.setAggregationResults(accumulators, newRow.row)
+    }
 
     // update accumulators
     state.update(accumulators)
 
-    out.collect(output)
-  }
+    // if previousRow is not null, do retraction process
+    if (generateRetraction && !firstRow) {
+      if (prevRow.row.equals(newRow.row)) {
+        // ignore same newRow
+        return
+      } else {
+        // retract previous row
+        out.collect(prevRow)
+      }
+    }
 
+    out.collect(newRow)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
index ec9b654..711cc05 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
@@ -22,6 +22,7 @@ import java.lang.Iterable
 import org.apache.flink.types.Row
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.util.Collector
 
 /**
@@ -39,17 +40,17 @@ class IncrementalAggregateAllTimeWindowFunction(
   extends IncrementalAggregateAllWindowFunction[TimeWindow](
     finalRowArity) {
 
-  private var collector: TimeWindowPropertyCollector = _
+  private var collector: CRowTimeWindowPropertyCollector = _
 
   override def open(parameters: Configuration): Unit = {
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    collector = new CRowTimeWindowPropertyCollector(windowStartPos, windowEndPos)
     super.open(parameters)
   }
 
   override def apply(
       window: TimeWindow,
       records: Iterable[Row],
-      out: Collector[Row]): Unit = {
+      out: Collector[CRow]): Unit = {
 
     // set collector and window
     collector.wrappedCollector = out

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
index f92be92..c190785 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
@@ -23,6 +23,7 @@ import org.apache.flink.types.Row
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
 import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.util.Collector
 
 /**
@@ -32,12 +33,12 @@ import org.apache.flink.util.Collector
   */
 class IncrementalAggregateAllWindowFunction[W <: Window](
     private val finalRowArity: Int)
-  extends RichAllWindowFunction[Row, Row, W] {
+  extends RichAllWindowFunction[Row, CRow, W] {
 
-  private var output: Row = _
+  private var output: CRow = _
 
   override def open(parameters: Configuration): Unit = {
-    output = new Row(finalRowArity)
+    output = new CRow(new Row(finalRowArity), true)
   }
 
   /**
@@ -47,7 +48,7 @@ class IncrementalAggregateAllWindowFunction[W <: Window](
   override def apply(
       window: W,
       records: Iterable[Row],
-      out: Collector[Row]): Unit = {
+      out: Collector[CRow]): Unit = {
 
     val iterator = records.iterator
 
@@ -55,7 +56,7 @@ class IncrementalAggregateAllWindowFunction[W <: Window](
       val record = iterator.next()
       var i = 0
       while (i < record.getArity) {
-        output.setField(i, record.getField(i))
+        output.row.setField(i, record.getField(i))
         i += 1
       }
       out.collect(output)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
index dccb4f6..809bbfd 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.types.Row
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.util.Collector
 
 /**
@@ -43,10 +44,10 @@ class IncrementalAggregateTimeWindowFunction(
     numAggregates,
     finalRowArity) {
 
-  private var collector: TimeWindowPropertyCollector = _
+  private var collector: CRowTimeWindowPropertyCollector = _
 
   override def open(parameters: Configuration): Unit = {
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    collector = new CRowTimeWindowPropertyCollector(windowStartPos, windowEndPos)
     super.open(parameters)
   }
 
@@ -54,7 +55,7 @@ class IncrementalAggregateTimeWindowFunction(
       key: Tuple,
       window: TimeWindow,
       records: Iterable[Row],
-      out: Collector[Row]): Unit = {
+      out: Collector[CRow]): Unit = {
 
     // set collector and window
     collector.wrappedCollector = out

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
index 983efb3..7e9d738 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
@@ -24,6 +24,7 @@ import org.apache.flink.types.Row
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
 import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.util.Collector
 
 /**
@@ -37,12 +38,12 @@ class IncrementalAggregateWindowFunction[W <: Window](
     private val numGroupingKey: Int,
     private val numAggregates: Int,
     private val finalRowArity: Int)
-  extends RichWindowFunction[Row, Row, Tuple, W] {
+  extends RichWindowFunction[Row, CRow, Tuple, W] {
 
-  private var output: Row = _
+  private var output: CRow = _
 
   override def open(parameters: Configuration): Unit = {
-    output = new Row(finalRowArity)
+    output = new CRow(new Row(finalRowArity), true)
   }
 
   /**
@@ -53,7 +54,7 @@ class IncrementalAggregateWindowFunction[W <: Window](
       key: Tuple,
       window: W,
       records: Iterable[Row],
-      out: Collector[Row]): Unit = {
+      out: Collector[CRow]): Unit = {
 
     val iterator = records.iterator
 
@@ -62,12 +63,12 @@ class IncrementalAggregateWindowFunction[W <: Window](
 
       var i = 0
       while (i < numGroupingKey) {
-        output.setField(i, key.getField(i))
+        output.row.setField(i, key.getField(i))
         i += 1
       }
       i = 0
       while (i < numAggregates) {
-        output.setField(numGroupingKey + i, record.getField(i))
+        output.row.setField(numGroupingKey + i, record.getField(i))
         i += 1
       }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
index b63eb81..3fb506f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
@@ -31,7 +31,8 @@ import org.apache.flink.api.java.typeutils.ListTypeInfo
 import java.util.{ArrayList, List => JList}
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
 
 /**
@@ -47,10 +48,10 @@ class ProcTimeBoundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
     precedingTimeBoundary: Long,
     aggregatesTypeInfo: RowTypeInfo,
-    inputType: TypeInformation[Row])
-  extends ProcessFunction[Row, Row]
+    inputType: TypeInformation[CRow])
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
-  private var output: Row = _
+  private var output: CRow = _
   private var accumulatorState: ValueState[Row] = _
   private var rowMapState: MapState[Long, JList[Row]] = _
 
@@ -66,11 +67,12 @@ class ProcTimeBoundedRangeOver(
       genAggregations.code)
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
 
     // We keep the elements received in a MapState indexed based on their ingestion time
     val rowListTypeInfo: TypeInformation[JList[Row]] =
-      new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]]
+      new ListTypeInfo[Row](inputType.asInstanceOf[CRowTypeInfo].rowType)
+        .asInstanceOf[TypeInformation[JList[Row]]]
     val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
       new MapStateDescriptor[Long, JList[Row]]("rowmapstate",
         BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo)
@@ -82,9 +84,9 @@ class ProcTimeBoundedRangeOver(
   }
 
   override def processElement(
-    input: Row,
-    ctx: ProcessFunction[Row, Row]#Context,
-    out: Collector[Row]): Unit = {
+    input: CRow,
+    ctx: ProcessFunction[CRow, CRow]#Context,
+    out: Collector[CRow]): Unit = {
 
     val currentTime = ctx.timerService.currentProcessingTime
     // buffer the event incoming event
@@ -97,15 +99,15 @@ class ProcTimeBoundedRangeOver(
       // register timer to process event once the current millisecond passed
       ctx.timerService.registerProcessingTimeTimer(currentTime + 1)
     }
-    rowList.add(input)
+    rowList.add(input.row)
     rowMapState.put(currentTime, rowList)
 
   }
 
   override def onTimer(
     timestamp: Long,
-    ctx: ProcessFunction[Row, Row]#OnTimerContext,
-    out: Collector[Row]): Unit = {
+    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+    out: Collector[CRow]): Unit = {
 
     // we consider the original timestamp of events that have registered this time trigger 1 ms ago
     val currentTime = timestamp - 1
@@ -166,10 +168,10 @@ class ProcTimeBoundedRangeOver(
       val input = currentElements.get(iElemenets)
 
       // set the fields of the last event to carry on with the aggregates
-      function.setForwardedFields(input, output)
+      function.setForwardedFields(input, output.row)
 
       // add the accumulators values to result
-      function.setAggregationResults(accumulators, output)
+      function.setAggregationResults(accumulators, output.row)
       out.collect(output)
       iElemenets += 1
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala
index 31cfd73..0c7f44e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala
@@ -33,7 +33,8 @@ import org.apache.flink.api.java.typeutils.ListTypeInfo
 import java.util.{List => JList}
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
 
 /**
@@ -48,15 +49,15 @@ class ProcTimeBoundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     precedingOffset: Long,
     aggregatesTypeInfo: RowTypeInfo,
-    inputType: TypeInformation[Row])
-  extends ProcessFunction[Row, Row]
+    inputType: TypeInformation[CRow])
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
 
   Preconditions.checkArgument(precedingOffset > 0)
 
   private var accumulatorState: ValueState[Row] = _
   private var rowMapState: MapState[Long, JList[Row]] = _
-  private var output: Row = _
+  private var output: CRow = _
   private var counterState: ValueState[Long] = _
   private var smallestTsState: ValueState[Long] = _
 
@@ -73,13 +74,14 @@ class ProcTimeBoundedRowsOver(
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
 
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
     // We keep the elements received in a Map state keyed
     // by the ingestion time in the operator.
     // we also keep counter of processed elements
     // and timestamp of oldest element
     val rowListTypeInfo: TypeInformation[JList[Row]] =
-      new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]]
+      new ListTypeInfo[Row](inputType.asInstanceOf[CRowTypeInfo].rowType)
+        .asInstanceOf[TypeInformation[JList[Row]]]
 
     val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
       new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState",
@@ -100,9 +102,11 @@ class ProcTimeBoundedRowsOver(
   }
 
   override def processElement(
-    input: Row,
-    ctx: ProcessFunction[Row, Row]#Context,
-    out: Collector[Row]): Unit = {
+    inputC: CRow,
+    ctx: ProcessFunction[CRow, CRow]#Context,
+    out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
     val currentTime = ctx.timerService.currentProcessingTime
 
@@ -154,11 +158,11 @@ class ProcTimeBoundedRowsOver(
     }
 
     // copy forwarded fields in output row
-    function.setForwardedFields(input, output)
+    function.setForwardedFields(input, output.row)
 
     // accumulate current row and set aggregate in output row
     function.accumulate(accumulators, input)
-    function.setAggregationResults(accumulators, output)
+    function.setAggregationResults(accumulators, output.row)
 
     // update map state, accumulator state, counter and timestamp
     val currentTimeState = rowMapState.get(currentTime)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala
index 75209db..8a23132 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala
@@ -23,9 +23,10 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.state.{FunctionInitializationContext, FunctionSnapshotContext}
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction
 import org.apache.flink.streaming.api.functions.ProcessFunction
-import org.apache.flink.types.Row
 import org.apache.flink.util.Collector
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.types.Row
 import org.slf4j.LoggerFactory
 
 /**
@@ -37,12 +38,12 @@ import org.slf4j.LoggerFactory
 class ProcTimeUnboundedNonPartitionedOver(
     genAggregations: GeneratedAggregationsFunction,
     aggregationStateType: RowTypeInfo)
-  extends ProcessFunction[Row, Row]
+  extends ProcessFunction[CRow, CRow]
     with CheckpointedFunction
     with Compiler[GeneratedAggregations] {
 
   private var accumulators: Row = _
-  private var output: Row = _
+  private var output: CRow = _
   private var state: ListState[Row] = _
   val LOG = LoggerFactory.getLogger(this.getClass)
 
@@ -58,7 +59,7 @@ class ProcTimeUnboundedNonPartitionedOver(
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
 
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
     if (null == accumulators) {
       val it = state.get().iterator()
       if (it.hasNext) {
@@ -70,14 +71,16 @@ class ProcTimeUnboundedNonPartitionedOver(
   }
 
   override def processElement(
-    input: Row,
-    ctx: ProcessFunction[Row, Row]#Context,
-    out: Collector[Row]): Unit = {
+      inputC: CRow,
+      ctx: ProcessFunction[CRow, CRow]#Context,
+      out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
-    function.setForwardedFields(input, output)
+    function.setForwardedFields(input, output.row)
 
     function.accumulate(accumulators, input)
-    function.setAggregationResults(accumulators, output)
+    function.setAggregationResults(accumulators, output.row)
 
     out.collect(output)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala
index 9baa6a3..847c1bf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala
@@ -24,7 +24,8 @@ import org.apache.flink.util.Collector
 import org.apache.flink.api.common.state.ValueStateDescriptor
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.api.common.state.ValueState
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.CRow
 import org.slf4j.LoggerFactory
 
 /**
@@ -36,10 +37,10 @@ import org.slf4j.LoggerFactory
 class ProcTimeUnboundedPartitionedOver(
     genAggregations: GeneratedAggregationsFunction,
     aggregationStateType: RowTypeInfo)
-  extends ProcessFunction[Row, Row]
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
 
-  private var output: Row = _
+  private var output: CRow = _
   private var state: ValueState[Row] = _
   val LOG = LoggerFactory.getLogger(this.getClass)
   private var function: GeneratedAggregations = _
@@ -54,16 +55,18 @@ class ProcTimeUnboundedPartitionedOver(
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
 
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
     val stateDescriptor: ValueStateDescriptor[Row] =
       new ValueStateDescriptor[Row]("overState", aggregationStateType)
     state = getRuntimeContext.getState(stateDescriptor)
   }
 
   override def processElement(
-    input: Row,
-    ctx: ProcessFunction[Row, Row]#Context,
-    out: Collector[Row]): Unit = {
+    inputC: CRow,
+    ctx: ProcessFunction[CRow, CRow]#Context,
+    out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
     var accumulators = state.value()
 
@@ -71,13 +74,12 @@ class ProcTimeUnboundedPartitionedOver(
       accumulators = function.createAccumulators()
     }
 
-    function.setForwardedFields(input, output)
+    function.setForwardedFields(input, output.row)
 
     function.accumulate(accumulators, input)
-    function.setAggregationResults(accumulators, output)
+    function.setAggregationResults(accumulators, output.row)
 
     state.update(accumulators)
-
     out.collect(output)
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
index ef97e71..4020d44 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
@@ -17,14 +17,15 @@
  */
 package org.apache.flink.table.runtime.aggregate
 
-import java.util.{List => JList, ArrayList => JArrayList}
+import java.util.{ArrayList => JArrayList, List => JList}
 
 import org.apache.flink.api.common.state._
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo}
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.functions.ProcessFunction
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.apache.flink.types.Row
 import org.apache.flink.util.{Collector, Preconditions}
 import org.slf4j.LoggerFactory
@@ -39,15 +40,15 @@ import org.slf4j.LoggerFactory
  */
 class RowTimeBoundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
-    aggregationStateType: TypeInformation[Row],
-    inputRowType: TypeInformation[Row],
+    aggregationStateType: RowTypeInfo,
+    inputRowType: CRowTypeInfo,
     precedingOffset: Long)
-  extends ProcessFunction[Row, Row]
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
   Preconditions.checkNotNull(aggregationStateType)
   Preconditions.checkNotNull(precedingOffset)
 
-  private var output: Row = _
+  private var output: CRow = _
 
   // the state which keeps the last triggering timestamp
   private var lastTriggeringTsState: ValueState[Long] = _
@@ -74,7 +75,7 @@ class RowTimeBoundedRangeOver(
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
 
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
 
     val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
       new ValueStateDescriptor[Long]("lastTriggeringTsState", classOf[Long])
@@ -86,7 +87,8 @@ class RowTimeBoundedRangeOver(
 
     val keyTypeInformation: TypeInformation[Long] =
       BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
-    val valueTypeInformation: TypeInformation[JList[Row]] = new ListTypeInfo[Row](inputRowType)
+    val valueTypeInformation: TypeInformation[JList[Row]] =
+      new ListTypeInfo[Row](inputRowType.asInstanceOf[CRowTypeInfo].rowType)
 
     val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
       new MapStateDescriptor[Long, JList[Row]](
@@ -98,9 +100,11 @@ class RowTimeBoundedRangeOver(
   }
 
   override def processElement(
-    input: Row,
-    ctx: ProcessFunction[Row, Row]#Context,
-    out: Collector[Row]): Unit = {
+    inputC: CRow,
+    ctx: ProcessFunction[CRow, CRow]#Context,
+    out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
     // triggering timestamp for trigger calculation
     val triggeringTs = ctx.timestamp
@@ -125,8 +129,8 @@ class RowTimeBoundedRangeOver(
 
   override def onTimer(
     timestamp: Long,
-    ctx: ProcessFunction[Row, Row]#OnTimerContext,
-    out: Collector[Row]): Unit = {
+    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+    out: Collector[CRow]): Unit = {
     // gets all window data from state for the calculation
     val inputs: JList[Row] = dataState.get(timestamp)
 
@@ -172,13 +176,13 @@ class RowTimeBoundedRangeOver(
       }
 
       // set aggregate in output row
-      function.setAggregationResults(accumulators, output)
+      function.setAggregationResults(accumulators, output.row)
 
       // copy forwarded fields to output row and emit output row
       dataListIndex = 0
       while (dataListIndex < inputs.size()) {
         aggregatesIndex = 0
-        function.setForwardedFields(inputs.get(dataListIndex), output)
+        function.setForwardedFields(inputs.get(dataListIndex), output.row)
         out.collect(output)
         dataListIndex += 1
       }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
index 7169cf7..5ec6ec7 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
@@ -27,7 +27,8 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.types.Row
 import org.apache.flink.util.{Collector, Preconditions}
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
 
 /**
@@ -41,15 +42,15 @@ import org.slf4j.LoggerFactory
 class RowTimeBoundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     aggregationStateType: RowTypeInfo,
-    inputRowType: TypeInformation[Row],
+    inputRowType: CRowTypeInfo,
     precedingOffset: Long)
-  extends ProcessFunction[Row, Row]
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
 
   Preconditions.checkNotNull(aggregationStateType)
   Preconditions.checkNotNull(precedingOffset)
 
-  private var output: Row = _
+  private var output: CRow = _
 
   // the state which keeps the last triggering timestamp
   private var lastTriggeringTsState: ValueState[Long] = _
@@ -79,7 +80,7 @@ class RowTimeBoundedRowsOver(
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
 
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
 
     val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
       new ValueStateDescriptor[Long]("lastTriggeringTsState", classOf[Long])
@@ -95,7 +96,8 @@ class RowTimeBoundedRowsOver(
 
     val keyTypeInformation: TypeInformation[Long] =
       BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
-    val valueTypeInformation: TypeInformation[JList[Row]] = new ListTypeInfo[Row](inputRowType)
+    val valueTypeInformation: TypeInformation[JList[Row]] =
+      new ListTypeInfo[Row](inputRowType.asInstanceOf[CRowTypeInfo].rowType)
 
     val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
       new MapStateDescriptor[Long, JList[Row]](
@@ -107,9 +109,11 @@ class RowTimeBoundedRowsOver(
   }
 
   override def processElement(
-    input: Row,
-    ctx: ProcessFunction[Row, Row]#Context,
-    out: Collector[Row]): Unit = {
+    inputC: CRow,
+    ctx: ProcessFunction[CRow, CRow]#Context,
+    out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
     // triggering timestamp for trigger calculation
     val triggeringTs = ctx.timestamp
@@ -134,8 +138,8 @@ class RowTimeBoundedRowsOver(
 
   override def onTimer(
     timestamp: Long,
-    ctx: ProcessFunction[Row, Row]#OnTimerContext,
-    out: Collector[Row]): Unit = {
+    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+    out: Collector[CRow]): Unit = {
 
     // gets all window data from state for the calculation
     val inputs: JList[Row] = dataState.get(timestamp)
@@ -189,7 +193,7 @@ class RowTimeBoundedRowsOver(
         }
 
         // copy forwarded fields to output row
-        function.setForwardedFields(input, output)
+        function.setForwardedFields(input, output.row)
 
         // retract old row from accumulators
         if (null != retractRow) {
@@ -198,7 +202,7 @@ class RowTimeBoundedRowsOver(
 
         // accumulate current row and set aggregate in output row
         function.accumulate(accumulators, input)
-        function.setAggregationResults(accumulators, output)
+        function.setAggregationResults(accumulators, output.row)
         i += 1
 
         out.collect(output)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala
index 525d4d7..3e2a811 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala
@@ -28,7 +28,8 @@ import org.apache.flink.util.{Collector, Preconditions}
 import org.apache.flink.api.common.state._
 import org.apache.flink.api.java.typeutils.ListTypeInfo
 import org.apache.flink.streaming.api.operators.TimestampedCollector
-import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
+import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
 
 
@@ -42,11 +43,11 @@ import org.slf4j.LoggerFactory
 abstract class RowTimeUnboundedOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType: TypeInformation[Row],
-    inputType: TypeInformation[Row])
-  extends ProcessFunction[Row, Row]
+    inputType: TypeInformation[CRow])
+  extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
 
-  protected var output: Row = _
+  protected var output: CRow = _
   // state to hold the accumulators of the aggregations
   private var accumulatorState: ValueState[Row] = _
   // state to hold rows until the next watermark arrives
@@ -67,7 +68,7 @@ abstract class RowTimeUnboundedOver(
     LOG.debug("Instantiating AggregateHelper.")
     function = clazz.newInstance()
 
-    output = function.createOutputRow()
+    output = new CRow(function.createOutputRow(), true)
     sortedTimestamps = new util.LinkedList[Long]()
 
     // initialize accumulator state
@@ -76,7 +77,8 @@ abstract class RowTimeUnboundedOver(
     accumulatorState = getRuntimeContext.getState[Row](accDescriptor)
 
     // initialize row state
-    val rowListTypeInfo: TypeInformation[JList[Row]] = new ListTypeInfo[Row](inputType)
+    val rowListTypeInfo: TypeInformation[JList[Row]] =
+      new ListTypeInfo[Row](inputType.asInstanceOf[CRowTypeInfo].rowType)
     val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
       new MapStateDescriptor[Long, JList[Row]]("rowmapstate",
         BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo)
@@ -87,15 +89,17 @@ abstract class RowTimeUnboundedOver(
     * Puts an element from the input stream into state if it is not late.
     * Registers a timer for the next watermark.
     *
-    * @param input The input value.
+    * @param inputC The input value.
     * @param ctx   The ctx to register timer or get current time
     * @param out   The collector for returning result values.
     *
     */
   override def processElement(
-     input: Row,
-     ctx:  ProcessFunction[Row, Row]#Context,
-     out: Collector[Row]): Unit = {
+     inputC: CRow,
+     ctx:  ProcessFunction[CRow, CRow]#Context,
+     out: Collector[CRow]): Unit = {
+
+    val input = inputC.row
 
     val timestamp = ctx.timestamp()
     val curWatermark = ctx.timerService().currentWatermark()
@@ -126,11 +130,11 @@ abstract class RowTimeUnboundedOver(
     */
   override def onTimer(
       timestamp: Long,
-      ctx: ProcessFunction[Row, Row]#OnTimerContext,
-      out: Collector[Row]): Unit = {
+      ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+      out: Collector[CRow]): Unit = {
 
-    Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
-    val collector = out.asInstanceOf[TimestampedCollector[Row]]
+    Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[CRow]])
+    val collector = out.asInstanceOf[TimestampedCollector[CRow]]
 
     val keyIterator = rowMapState.keys.iterator
     if (keyIterator.hasNext) {
@@ -206,7 +210,7 @@ abstract class RowTimeUnboundedOver(
   def processElementsWithSameTimestamp(
     curRowList: JList[Row],
     lastAccumulator: Row,
-    out: Collector[Row]): Unit
+    out: Collector[CRow]): Unit
 
 }
 
@@ -217,7 +221,7 @@ abstract class RowTimeUnboundedOver(
 class RowTimeUnboundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType: TypeInformation[Row],
-    inputType: TypeInformation[Row])
+    inputType: TypeInformation[CRow])
   extends RowTimeUnboundedOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType,
@@ -226,7 +230,7 @@ class RowTimeUnboundedRowsOver(
   override def processElementsWithSameTimestamp(
     curRowList: JList[Row],
     lastAccumulator: Row,
-    out: Collector[Row]): Unit = {
+    out: Collector[CRow]): Unit = {
 
     var i = 0
     while (i < curRowList.size) {
@@ -234,11 +238,11 @@ class RowTimeUnboundedRowsOver(
 
       var j = 0
       // copy forwarded fields to output row
-      function.setForwardedFields(curRow, output)
+      function.setForwardedFields(curRow, output.row)
 
       // update accumulators and copy aggregates to output row
       function.accumulate(lastAccumulator, curRow)
-      function.setAggregationResults(lastAccumulator, output)
+      function.setAggregationResults(lastAccumulator, output.row)
       // emit output row
       out.collect(output)
       i += 1
@@ -255,7 +259,7 @@ class RowTimeUnboundedRowsOver(
 class RowTimeUnboundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType: TypeInformation[Row],
-    inputType: TypeInformation[Row])
+    inputType: TypeInformation[CRow])
   extends RowTimeUnboundedOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType,
@@ -264,7 +268,7 @@ class RowTimeUnboundedRangeOver(
   override def processElementsWithSameTimestamp(
     curRowList: JList[Row],
     lastAccumulator: Row,
-    out: Collector[Row]): Unit = {
+    out: Collector[CRow]): Unit = {
 
     var i = 0
     // all same timestamp data should have same aggregation value.
@@ -281,10 +285,10 @@ class RowTimeUnboundedRangeOver(
       val curRow = curRowList.get(i)
 
       // copy forwarded fields to output row
-      function.setForwardedFields(curRow, output)
+      function.setForwardedFields(curRow, output.row)
 
       //copy aggregates to output row
-      function.setAggregationResults(lastAccumulator, output)
+      function.setAggregationResults(lastAccumulator, output.row)
       out.collect(output)
       i += 1
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
index 9502607..0c8ae00 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.runtime.aggregate
 
 import org.apache.calcite.runtime.SqlFunctions
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.types.Row
 import org.apache.flink.util.Collector
 
@@ -26,29 +27,48 @@ import org.apache.flink.util.Collector
   * Adds TimeWindow properties to specified fields of a row before it emits the row to a wrapped
   * collector.
   */
-class TimeWindowPropertyCollector(windowStartOffset: Option[Int], windowEndOffset: Option[Int])
-    extends Collector[Row] {
+abstract class TimeWindowPropertyCollector[T](
+    windowStartOffset: Option[Int],
+    windowEndOffset: Option[Int])
+  extends Collector[T] {
 
-  var wrappedCollector: Collector[Row] = _
+  var wrappedCollector: Collector[T] = _
+  var output: Row = _
   var windowStart:Long = _
   var windowEnd:Long = _
 
-  override def collect(record: Row): Unit = {
+  def getRow(record: T): Row
 
-    val lastFieldPos = record.getArity - 1
+  override def collect(record: T): Unit = {
+
+    output = getRow(record)
+    val lastFieldPos = output.getArity - 1
 
     if (windowStartOffset.isDefined) {
-      record.setField(
+      output.setField(
         lastFieldPos + windowStartOffset.get,
         SqlFunctions.internalToTimestamp(windowStart))
     }
     if (windowEndOffset.isDefined) {
-      record.setField(
+      output.setField(
         lastFieldPos + windowEndOffset.get,
         SqlFunctions.internalToTimestamp(windowEnd))
     }
+
     wrappedCollector.collect(record)
   }
 
   override def close(): Unit = wrappedCollector.close()
 }
+
+class RowTimeWindowPropertyCollector(startOffset: Option[Int], endOffset: Option[Int])
+  extends TimeWindowPropertyCollector[Row](startOffset, endOffset) {
+
+  override def getRow(record: Row): Row = record
+}
+
+class CRowTimeWindowPropertyCollector(startOffset: Option[Int], endOffset: Option[Int])
+  extends TimeWindowPropertyCollector[CRow](startOffset, endOffset) {
+
+  override def getRow(record: CRow): Row = record.row
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
new file mode 100644
index 0000000..ec73fa6
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala
@@ -0,0 +1,59 @@
+/*
+ * 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.table.runtime.io
+
+import org.apache.flink.api.common.io.{GenericInputFormat, NonParallelInput}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.core.io.GenericInputSplit
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.slf4j.LoggerFactory
+
+class CRowValuesInputFormat(
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[CRow])
+  extends GenericInputFormat[CRow]
+  with NonParallelInput
+  with ResultTypeQueryable[CRow]
+  with Compiler[GenericInputFormat[Row]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var format: GenericInputFormat[Row] = _
+
+  override def open(split: GenericInputSplit): Unit = {
+    LOG.debug(s"Compiling GenericInputFormat: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating GenericInputFormat.")
+    format = clazz.newInstance()
+  }
+
+  override def reachedEnd(): Boolean = format.reachedEnd()
+
+  override def nextRecord(reuse: CRow): CRow = {
+    reuse.row = format.nextRecord(reuse.row)
+    reuse.change = true
+    reuse
+  }
+
+  override def getProducedType: TypeInformation[CRow] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
index 1a339e6..d536b39 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
@@ -23,20 +23,21 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
 import org.apache.flink.table.codegen.Compiler
 import org.apache.flink.core.io.GenericInputSplit
+import org.apache.flink.types.Row
 import org.slf4j.LoggerFactory
 
-class ValuesInputFormat[OUT](
+class ValuesInputFormat(
     name: String,
     code: String,
-    @transient returnType: TypeInformation[OUT])
-  extends GenericInputFormat[OUT]
+    @transient returnType: TypeInformation[Row])
+  extends GenericInputFormat[Row]
   with NonParallelInput
-  with ResultTypeQueryable[OUT]
-  with Compiler[GenericInputFormat[OUT]] {
+  with ResultTypeQueryable[Row]
+  with Compiler[GenericInputFormat[Row]] {
 
   val LOG = LoggerFactory.getLogger(this.getClass)
 
-  private var format: GenericInputFormat[OUT] = _
+  private var format: GenericInputFormat[Row] = _
 
   override def open(split: GenericInputSplit): Unit = {
     LOG.debug(s"Compiling GenericInputFormat: $name \n\n Code:\n$code")
@@ -47,7 +48,7 @@ class ValuesInputFormat[OUT](
 
   override def reachedEnd(): Boolean = format.reachedEnd()
 
-  override def nextRecord(reuse: OUT): OUT = format.nextRecord(reuse)
+  override def nextRecord(reuse: Row): Row = format.nextRecord(reuse)
 
-  override def getProducedType: TypeInformation[OUT] = returnType
+  override def getProducedType: TypeInformation[Row] = returnType
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala
new file mode 100644
index 0000000..25ec8c4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.table.runtime.types
+
+import org.apache.flink.types.Row
+
+/**
+  * Wrapper for a [[Row]] to add retraction information.
+  *
+  * If [[change]] is true, the [[CRow]] is an accumulate message, if it is false it is a
+  * retraction message.
+  *
+  * @param row The wrapped [[Row]].
+  * @param change true for an accumulate message, false for a retraction message.
+  */
+class CRow(var row: Row, var change: Boolean) {
+
+  def this() {
+    this(null, true)
+  }
+
+  override def toString: String = s"${if(change) "+" else "-"}$row"
+
+  override def equals(other: scala.Any): Boolean = {
+    val otherCRow = other.asInstanceOf[CRow]
+    row.equals(otherCRow.row) && change == otherCRow.change
+  }
+}
+
+object CRow {
+
+  def apply(): CRow = {
+    new CRow()
+  }
+
+  def apply(row: Row, change: Boolean): CRow = {
+    new CRow(row, change)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowComparator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowComparator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowComparator.scala
new file mode 100644
index 0000000..d848c65
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowComparator.scala
@@ -0,0 +1,83 @@
+/*
+ * 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.table.runtime.types
+
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.core.memory.{DataInputView, DataOutputView, MemorySegment}
+import org.apache.flink.types.Row
+
+class CRowComparator(val rowComp: TypeComparator[Row]) extends TypeComparator[CRow] {
+
+  override def hash(record: CRow): Int = rowComp.hash(record.row)
+
+  override def setReference(toCompare: CRow): Unit = rowComp.setReference(toCompare.row)
+
+  override def equalToReference(candidate: CRow): Boolean = rowComp.equalToReference(candidate.row)
+
+  override def compareToReference(otherComp: TypeComparator[CRow]): Int = {
+    val otherCRowComp = otherComp.asInstanceOf[CRowComparator]
+    rowComp.compareToReference(otherCRowComp.rowComp)
+  }
+
+  override def compare(first: CRow, second: CRow): Int = {
+    rowComp.compare(first.row, second.row)
+  }
+
+  override def compareSerialized(firstSource: DataInputView, secondSource: DataInputView): Int = {
+    rowComp.compareSerialized(firstSource, secondSource)
+  }
+
+  override def supportsNormalizedKey(): Boolean = rowComp.supportsNormalizedKey()
+
+  override def supportsSerializationWithKeyNormalization(): Boolean =
+    rowComp.supportsSerializationWithKeyNormalization()
+
+  override def getNormalizeKeyLen: Int = rowComp.getNormalizeKeyLen
+
+  override def isNormalizedKeyPrefixOnly(keyBytes: Int): Boolean =
+    rowComp.isNormalizedKeyPrefixOnly(keyBytes)
+
+  override def putNormalizedKey(
+      record: CRow,
+      target: MemorySegment,
+      offset: Int,
+      numBytes: Int): Unit = rowComp.putNormalizedKey(record.row, target, offset, numBytes)
+
+  override def writeWithKeyNormalization(record: CRow, target: DataOutputView): Unit = {
+    rowComp.writeWithKeyNormalization(record.row, target)
+    target.writeBoolean(record.change)
+  }
+
+  override def readWithKeyDenormalization(reuse: CRow, source: DataInputView): CRow = {
+    val row = rowComp.readWithKeyDenormalization(reuse.row, source)
+    reuse.row = row
+    reuse.change = source.readBoolean()
+    reuse
+  }
+
+  override def invertNormalizedKey(): Boolean = rowComp.invertNormalizedKey()
+
+  override def duplicate(): TypeComparator[CRow] = new CRowComparator(rowComp.duplicate())
+
+  override def extractKeys(record: scala.Any, target: Array[AnyRef], index: Int): Int =
+    rowComp.extractKeys(record.asInstanceOf[CRow].row, target, index)
+
+  override def getFlatComparators: Array[TypeComparator[_]] =
+    rowComp.getFlatComparators
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala
new file mode 100644
index 0000000..1f56a98
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala
@@ -0,0 +1,78 @@
+/*
+ * 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.table.runtime.types
+
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.core.memory.{DataInputView, DataOutputView}
+import org.apache.flink.types.Row
+
+class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSerializer[CRow] {
+
+  override def isImmutableType: Boolean = false
+
+  override def duplicate(): TypeSerializer[CRow] = new CRowSerializer(rowSerializer.duplicate())
+
+  override def createInstance(): CRow = new CRow(rowSerializer.createInstance(), true)
+
+  override def copy(from: CRow): CRow = new CRow(rowSerializer.copy(from.row), from.change)
+
+  override def copy(from: CRow, reuse: CRow): CRow = {
+    rowSerializer.copy(from.row, reuse.row)
+    reuse.change = from.change
+    reuse
+  }
+
+  override def getLength: Int = -1
+
+  override def serialize(record: CRow, target: DataOutputView): Unit = {
+    rowSerializer.serialize(record.row, target)
+    target.writeBoolean(record.change)
+  }
+
+  override def deserialize(source: DataInputView): CRow = {
+    val row = rowSerializer.deserialize(source)
+    val change = source.readBoolean()
+    new CRow(row, change)
+  }
+
+  override def deserialize(reuse: CRow, source: DataInputView): CRow = {
+    rowSerializer.deserialize(reuse.row, source)
+    reuse.change = source.readBoolean()
+    reuse
+  }
+
+  override def copy(source: DataInputView, target: DataOutputView): Unit = {
+    rowSerializer.copy(source, target)
+    target.writeBoolean(source.readBoolean())
+  }
+
+  override def canEqual(obj: Any): Boolean = obj.isInstanceOf[CRowSerializer]
+
+  override def equals(obj: Any): Boolean = {
+
+    if (canEqual(obj)) {
+      val other = obj.asInstanceOf[CRowSerializer]
+      rowSerializer.equals(other.rowSerializer)
+    } else {
+      false
+    }
+  }
+
+  override def hashCode: Int = rowSerializer.hashCode() * 13
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowTypeInfo.scala
new file mode 100644
index 0000000..456207a
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowTypeInfo.scala
@@ -0,0 +1,98 @@
+/*
+ * 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.table.runtime.types
+
+import java.util
+
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.typeutils.{CompositeType, TypeComparator, TypeSerializer}
+import org.apache.flink.api.common.typeutils.CompositeType.{FlatFieldDescriptor, TypeComparatorBuilder}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+
+class CRowTypeInfo(val rowType: RowTypeInfo) extends CompositeType[CRow](classOf[CRow]) {
+
+  override def getFieldNames: Array[String] = rowType.getFieldNames
+
+  override def getFieldIndex(fieldName: String): Int = rowType.getFieldIndex(fieldName)
+
+  override def getTypeAt[X](fieldExpression: String): TypeInformation[X] =
+    rowType.getTypeAt(fieldExpression)
+
+  override def getTypeAt[X](pos: Int): TypeInformation[X] =
+    rowType.getTypeAt(pos)
+
+  override def getFlatFields(
+      fieldExpression: String,
+      offset: Int,
+      result: util.List[FlatFieldDescriptor]): Unit =
+    rowType.getFlatFields(fieldExpression, offset, result)
+
+  override def isBasicType: Boolean = rowType.isBasicType
+
+  override def isTupleType: Boolean = rowType.isTupleType
+
+  override def getArity: Int = rowType.getArity
+
+  override def getTotalFields: Int = rowType.getTotalFields
+
+  override def createSerializer(config: ExecutionConfig): TypeSerializer[CRow] =
+    new CRowSerializer(rowType.createSerializer(config))
+
+  // not implemented because we override createComparator
+  override protected def createTypeComparatorBuilder(): TypeComparatorBuilder[CRow] = null
+
+  override def createComparator(
+      logicalKeyFields: Array[Int],
+      orders: Array[Boolean],
+      logicalFieldOffset: Int,
+      config: ExecutionConfig): TypeComparator[CRow] = {
+
+    val rowComparator = rowType.createComparator(
+      logicalKeyFields,
+      orders,
+      logicalFieldOffset,
+      config)
+
+    new CRowComparator(rowComparator)
+  }
+
+  override def equals(obj: scala.Any): Boolean = {
+    if (this.canEqual(obj)) {
+      rowType.equals(obj.asInstanceOf[CRowTypeInfo].rowType)
+    } else {
+      false
+    }
+  }
+
+  override def canEqual(obj: scala.Any): Boolean = obj.isInstanceOf[CRowTypeInfo]
+
+}
+
+object CRowTypeInfo {
+
+  def apply(rowType: TypeInformation[Row]): CRowTypeInfo = {
+    rowType match {
+      case r: RowTypeInfo => new CRowTypeInfo(r)
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
index c37ee74..4a2fcdf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
@@ -25,6 +25,7 @@ import org.apache.flink.types.Row
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 /**
   * A simple [[TableSink]] to emit data as CSV files.
@@ -133,3 +134,4 @@ class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] {
     builder.mkString
   }
 }
+


[10/15] flink git commit: [FLINK-6091] [table] Implement and turn on retraction for non-windowed aggregates.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
index 5247685..60de1f1 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
@@ -22,9 +22,10 @@ import org.apache.flink.api.scala._
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.scala._
-import org.apache.flink.api.java.typeutils.{GenericTypeInfo, TupleTypeInfo, TypeExtractor}
+import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.expressions.{Alias, UnresolvedFieldReference}
+import org.apache.flink.table.runtime.types.CRowTypeInfo
 import org.apache.flink.table.utils.{MockTableEnvironment, TableTestBase}
 import org.apache.flink.table.utils.TableTestUtil.{batchTableNode, binaryNode, streamTableNode, term, unaryNode}
 import org.apache.flink.types.Row
@@ -40,9 +41,13 @@ class TableEnvironmentTest extends TableTestBase {
     STRING_TYPE_INFO,
     DOUBLE_TYPE_INFO)
 
-  val caseClassType = implicitly[TypeInformation[CClass]]
+  val rowType = new RowTypeInfo(INT_TYPE_INFO, STRING_TYPE_INFO,DOUBLE_TYPE_INFO)
 
-  val pojoType = TypeExtractor.createTypeInfo(classOf[PojoClass])
+  val cRowType = new CRowTypeInfo(rowType)
+
+  val caseClassType: TypeInformation[CClass] = implicitly[TypeInformation[CClass]]
+
+  val pojoType: TypeInformation[PojoClass] = TypeExtractor.createTypeInfo(classOf[PojoClass])
 
   val atomicType = INT_TYPE_INFO
 
@@ -57,6 +62,14 @@ class TableEnvironmentTest extends TableTestBase {
   }
 
   @Test
+  def testGetFieldInfoCRow(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(cRowType)
+
+    fieldInfo._1.zip(Array("f0", "f1", "f2")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
   def testGetFieldInfoCClass(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(caseClassType)
 
@@ -100,6 +113,20 @@ class TableEnvironmentTest extends TableTestBase {
   }
 
   @Test
+  def testGetFieldInfoCRowNames(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      cRowType,
+      Array(
+        UnresolvedFieldReference("name1"),
+        UnresolvedFieldReference("name2"),
+        UnresolvedFieldReference("name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
   def testGetFieldInfoCClassNames(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(
       caseClassType,
@@ -198,6 +225,45 @@ class TableEnvironmentTest extends TableTestBase {
   }
 
   @Test
+  def testGetFieldInfoCRowAlias1(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      cRowType,
+      Array(
+        Alias(UnresolvedFieldReference("f0"), "name1"),
+        Alias(UnresolvedFieldReference("f1"), "name2"),
+        Alias(UnresolvedFieldReference("f2"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoCRowAlias2(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      cRowType,
+      Array(
+        Alias(UnresolvedFieldReference("f2"), "name1"),
+        Alias(UnresolvedFieldReference("f0"), "name2"),
+        Alias(UnresolvedFieldReference("f1"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoCRowAlias3(): Unit = {
+    tEnv.getFieldInfo(
+      cRowType,
+      Array(
+        Alias(UnresolvedFieldReference("xxx"), "name1"),
+        Alias(UnresolvedFieldReference("yyy"), "name2"),
+        Alias(UnresolvedFieldReference("zzz"), "name3")
+      ))
+  }
+
+  @Test
   def testGetFieldInfoCClassAlias1(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(
       caseClassType,

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
index 57ee3b3..ebfac0a 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
@@ -28,6 +28,7 @@ import org.apache.flink.table.api.scala.batch.utils.{TableProgramsCollectionTest
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
 import org.apache.flink.types.Row
 import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.test.util.TestBaseUtils
 import org.junit._
 import org.junit.runner.RunWith
@@ -164,6 +165,25 @@ class TableEnvironmentITCase(
   }
 
   @Test
+  def testToDataSetWithTypeOfCRow(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env)
+      .toTable(tEnv, 'a, 'b, 'c)
+      .select('a, 'b, 'c)
+
+    val expected = "+1,1,Hi\n" + "+2,2,Hello\n" + "+3,2,Hello world\n" +
+      "+4,3,Hello world, how are you?\n" + "+5,3,I am fine.\n" + "+6,3,Luke Skywalker\n" +
+      "+7,4,Comment#1\n" + "+8,4,Comment#2\n" + "+9,4,Comment#3\n" + "+10,4,Comment#4\n" +
+      "+11,5,Comment#5\n" + "+12,5,Comment#6\n" + "+13,5,Comment#7\n" + "+14,5,Comment#8\n" +
+      "+15,5,Comment#9\n" + "+16,6,Comment#10\n" + "+17,6,Comment#11\n" + "+18,6,Comment#12\n" +
+      "+19,6,Comment#13\n" + "+20,6,Comment#14\n" + "+21,6,Comment#15\n"
+    val results = t.toDataSet[CRow].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
   def testToTableFromCaseClass(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env, config)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
new file mode 100644
index 0000000..dde7f89
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
@@ -0,0 +1,205 @@
+/*
+ * 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.table.api.scala.stream
+
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamingWithStateTestBase}
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.utils.TableFunc0
+
+import scala.collection.mutable
+
+/**
+  * tests for retraction
+  */
+class RetractionITCase extends StreamingWithStateTestBase {
+  // input data
+  val data = List(
+    ("Hello", 1),
+    ("word", 1),
+    ("Hello", 1),
+    ("bark", 1),
+    ("bark", 1),
+    ("bark", 1),
+    ("bark", 1),
+    ("bark", 1),
+    ("bark", 1),
+    ("flink", 1)
+  )
+
+  // keyed groupby + keyed groupby
+  @Test
+  def testWordCount(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+    env.setStateBackend(getStateBackend)
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'word, 'num)
+    val resultTable = table
+      .groupBy('word)
+      .select('word as 'word, 'num.sum as 'count)
+      .groupBy('count)
+      .select('count, 'word.count as 'frequency)
+
+    // to DataStream with CRow
+    val results = resultTable.toDataStream[CRow]
+    results.addSink(new StreamITCase.StringSinkWithCRow)
+    env.execute()
+
+    val expected = Seq("+1,1", "+1,2", "+1,1", "+2,1", "+1,2", "+1,1", "+2,2", "+2,1", "+3,1",
+      "+3,0", "+4,1", "+4,0", "+5,1", "+5,0", "+6,1", "+1,2")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  // keyed groupby + non-keyed groupby
+  @Test
+  def testGroupByAndNonKeyedGroupBy(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+    env.setStateBackend(getStateBackend)
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'word, 'num)
+    val resultTable = table
+      .groupBy('word)
+      .select('word as 'word, 'num.sum as 'count)
+      .select('count.sum)
+
+    val results = resultTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("1", "2", "1", "3", "4", "3", "5", "3", "6", "3", "7", "3", "8", "3", "9",
+      "10")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  // non-keyed groupby + keyed groupby
+  @Test
+  def testNonKeyedGroupByAndGroupBy(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+    env.setStateBackend(getStateBackend)
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'word, 'num)
+    val resultTable = table
+      .select('num.sum as 'count)
+      .groupBy('count)
+      .select('count, 'count.count)
+
+    val results = resultTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("1,1", "1,0", "2,1", "2,0", "3,1", "3,0", "4,1", "4,0", "5,1", "5,0", "6," +
+      "1", "6,0", "7,1", "7,0", "8,1", "8,0", "9,1", "9,0", "10,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  // test unique process, if the current output message of unbounded groupby equals the
+  // previous message, unbounded groupby will ignore the current one.
+  @Test
+  def testUniqueProcess(): Unit = {
+    // data input
+    val data = List(
+      (1, 1L),
+      (2, 2L),
+      (3, 3L),
+      (3, 3L),
+      (4, 1L),
+      (4, 0L),
+      (4, 0L),
+      (4, 0L),
+      (5, 1L),
+      (6, 6L),
+      (6, 6L),
+      (6, 6L),
+      (7, 8L)
+    )
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+    env.setStateBackend(getStateBackend)
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'pk, 'value)
+    val resultTable = table
+      .groupBy('pk)
+      .select('pk as 'pk, 'value.sum as 'sum)
+      .groupBy('sum)
+      .select('sum, 'pk.count as 'count)
+
+    val results = resultTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("1,1", "2,1", "3,1", "3,0", "6,1", "1,2", "1,3", "6,2", "6,1", "12,1","12," +
+      "0", "18,1", "8,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  // correlate should handle retraction messages correctly
+  @Test
+  def testCorrelate(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+    env.setStateBackend(getStateBackend)
+
+    val func0 = new TableFunc0
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'word, 'num)
+    val resultTable = table
+      .groupBy('word)
+      .select('word as 'word, 'num.sum as 'count)
+      .leftOuterJoin(func0('word))
+      .groupBy('count)
+      .select('count, 'word.count as 'frequency)
+
+    val results = resultTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "1,1", "1,2", "1,1", "2,1", "1,2", "1,1", "2,2", "2,1", "3,1", "3,0", "4,1", "4,0", "5,1",
+      "5,0", "6,1", "1,2")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
index cdc4329..c446d64 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
@@ -59,5 +59,5 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase {
 
     TestBaseUtils.compareResultsByLinesInMemory(expected, path)
   }
-
+  
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
index f826bba..6c75d53 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
@@ -25,6 +25,7 @@ import org.junit.Assert._
 
 import scala.collection.mutable
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+import org.apache.flink.table.runtime.types.CRow
 
 import scala.collection.JavaConverters._
 
@@ -44,7 +45,15 @@ object StreamITCase {
   final class StringSink extends RichSinkFunction[Row]() {
     def invoke(value: Row) {
       testResults.synchronized {
-        testResults += value.toString 
+        testResults += value.toString
+      }
+    }
+  }
+
+  final class StringSinkWithCRow extends RichSinkFunction[CRow]() {
+    def invoke(value: CRow) {
+      testResults.synchronized {
+        testResults += value.toString
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala
index 5e3e995..eadcfc8 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala
@@ -33,6 +33,7 @@ import org.apache.flink.table.codegen.GeneratedAggregationsFunction
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.functions.aggfunctions.{LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction}
 import org.apache.flink.table.runtime.aggregate.BoundedProcessingOverRangeProcessFunctionTest._
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.apache.flink.types.Row
 import org.junit.Test
 
@@ -41,13 +42,13 @@ class BoundedProcessingOverRangeProcessFunctionTest {
   @Test
   def testProcTimePartitionedOverRange(): Unit = {
 
-    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
+    val rT =  new CRowTypeInfo(new RowTypeInfo(Array[TypeInformation[_]](
       INT_TYPE_INFO,
       LONG_TYPE_INFO,
       INT_TYPE_INFO,
       STRING_TYPE_INFO,
       LONG_TYPE_INFO),
-      Array("a", "b", "c", "d", "e"))
+      Array("a", "b", "c", "d", "e")))
 
     val aggregates =
       Array(new LongMinWithRetractAggFunction,
@@ -183,14 +184,14 @@ class BoundedProcessingOverRangeProcessFunctionTest {
     val funcName = "BoundedOverAggregateHelper$33"
 
     val genAggFunction = GeneratedAggregationsFunction(funcName, funcCode)
-    val processFunction = new KeyedProcessOperator[String, Row, Row](
+    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new ProcTimeBoundedRangeOver(
         genAggFunction,
         1000,
         aggregationStateType,
         rT))
 
-    val testHarness = new KeyedOneInputStreamOperatorTestHarness[JInt, Row, Row](
+    val testHarness = new KeyedOneInputStreamOperatorTestHarness[JInt, CRow, CRow](
       processFunction,
       new TupleRowSelector(0),
       BasicTypeInfo.INT_TYPE_INFO)
@@ -201,26 +202,26 @@ class BoundedProcessingOverRangeProcessFunctionTest {
     testHarness.setProcessingTime(3)
     // key = 1
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0))
     // key = 2
     testHarness.processElement(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), 0))
+      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 0))
 
     // Time = 4
     testHarness.setProcessingTime(4)
     // key = 1
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 0))
     // key = 2
     testHarness.processElement(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), 0))
+      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 0))
 
     // Time = 5
     testHarness.setProcessingTime(5)
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0))
 
     // Time = 6
     testHarness.setProcessingTime(6)
@@ -229,33 +230,33 @@ class BoundedProcessingOverRangeProcessFunctionTest {
     testHarness.setProcessingTime(1002)
     // key = 1
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0))
     // key = 2
     testHarness.processElement(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), 0))
+      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0))
 
     // Time = 1003
     testHarness.setProcessingTime(1003)
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0))
 
     // Time = 1004
     testHarness.setProcessingTime(1004)
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0))
 
     // Time = 1005
     testHarness.setProcessingTime(1005)
     // key = 1
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), 0))
+      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 0))
     // key = 2
     testHarness.processElement(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), 0))
+      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 0))
 
     testHarness.setProcessingTime(1006)
 
@@ -264,34 +265,34 @@ class BoundedProcessingOverRangeProcessFunctionTest {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     // all elements at the same proc timestamp have the same value
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), 4))
-    expectedOutput.add(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), 4))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), 5))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), 5))
-    expectedOutput.add(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), 5))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), 6))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), 1003))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), 1003))
-    expectedOutput.add(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), 1003))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), 1004))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), 1005))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 4L: JLong, 10L: JLong), 1006))
-    expectedOutput.add(new StreamRecord(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 4L: JLong, 10L: JLong), 1006))
-    expectedOutput.add(new StreamRecord(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), 1006))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 4))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 4))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 5))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 5))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 5))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 6))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 1003))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 1003))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 1003))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 1004))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 1005))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 4L: JLong, 10L: JLong), true), 1006))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 4L: JLong, 10L: JLong), true), 1006))
+    expectedOutput.add(new StreamRecord(new CRow(
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), true), 1006))
 
     TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.",
         expectedOutput, result, new RowResultSortComparator(6))
@@ -304,7 +305,7 @@ class BoundedProcessingOverRangeProcessFunctionTest {
 object BoundedProcessingOverRangeProcessFunctionTest {
 
 /**
- * Return 0 for equal Rows and non zero for different rows
+ * Return 0 for equal CRows and non zero for different CRows
  */
 class RowResultSortComparator(indexCounter: Int) extends Comparator[Object] with Serializable {
 
@@ -314,8 +315,8 @@ class RowResultSortComparator(indexCounter: Int) extends Comparator[Object] with
         // watermark is not expected
          -1
        } else {
-        val row1 = o1.asInstanceOf[StreamRecord[Row]].getValue
-        val row2 = o2.asInstanceOf[StreamRecord[Row]].getValue
+        val row1 = o1.asInstanceOf[StreamRecord[CRow]].getValue
+        val row2 = o2.asInstanceOf[StreamRecord[CRow]].getValue
         row1.toString.compareTo(row2.toString)
       }
    }
@@ -325,10 +326,10 @@ class RowResultSortComparator(indexCounter: Int) extends Comparator[Object] with
  * Simple test class that returns a specified field as the selector function
  */
 class TupleRowSelector(
-    private val selectorField:Int) extends KeySelector[Row, Integer] {
+    private val selectorField:Int) extends KeySelector[CRow, Integer] {
 
-  override def getKey(value: Row): Integer = {
-    value.getField(selectorField).asInstanceOf[Integer]
+  override def getKey(value: CRow): Integer = {
+    value.row.getField(selectorField).asInstanceOf[Integer]
   }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowComparatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowComparatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowComparatorTest.scala
new file mode 100644
index 0000000..e574084
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowComparatorTest.scala
@@ -0,0 +1,61 @@
+/*
+ * 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.table.runtime.types
+
+import java.lang.{Integer => JInt, Long => JLong}
+
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+
+class CRowComparatorTest extends ComparatorTestBase[CRow] {
+
+  val rowType = new RowTypeInfo(
+    BasicTypeInfo.INT_TYPE_INFO,
+    BasicTypeInfo.STRING_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO
+  )
+
+  val cRowType = new CRowTypeInfo(rowType)
+
+  override protected def createComparator(asc: Boolean): TypeComparator[CRow] = {
+    cRowType.createComparator(
+      Array[Int](0, 2),
+      Array[Boolean](asc, asc),
+      0,
+      new ExecutionConfig
+    )
+  }
+
+  override protected def createSerializer(): TypeSerializer[CRow] =
+    cRowType.createSerializer(new ExecutionConfig)
+
+  override protected def getSortedTestData: Array[CRow] = Array[CRow](
+    new CRow(Row.of(new JInt(1), "Hello", new JLong(1L)), true),
+    new CRow(Row.of(new JInt(1), "Hello", new JLong(2L)), true),
+    new CRow(Row.of(new JInt(2), "Hello", new JLong(2L)), false),
+    new CRow(Row.of(new JInt(2), "Hello", new JLong(3L)), true),
+    new CRow(Row.of(new JInt(3), "World", new JLong(0L)), false),
+    new CRow(Row.of(new JInt(4), "Hello", new JLong(0L)), true),
+    new CRow(Row.of(new JInt(5), "Hello", new JLong(1L)), true),
+    new CRow(Row.of(new JInt(5), "Hello", new JLong(4L)), false)
+  )
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
index c5e13a1..79e957a 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
@@ -18,7 +18,9 @@
 
 package org.apache.flink.table.utils
 
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.tools.RuleSet
+import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.{Table, TableConfig, TableEnvironment}
 import org.apache.flink.table.sinks.TableSink
 import org.apache.flink.table.sources.TableSource
@@ -36,4 +38,10 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) {
   override protected def getBuiltInNormRuleSet: RuleSet = ???
 
   override protected def getBuiltInPhysicalOptRuleSet: RuleSet = ???
+
+  override protected def getConversionMapper[IN, OUT](
+      physicalTypeInfo: TypeInformation[IN],
+      logicalRowType: RelDataType,
+      requestedTypeInfo: TypeInformation[OUT],
+      functionName: String) = ???
 }


[13/15] flink git commit: [FLINK-6093] [table] Implement and turn on retraction for table sinks.

Posted by fh...@apache.org.
[FLINK-6093] [table] Implement and turn on retraction for table sinks.


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

Branch: refs/heads/master
Commit: bfed279f05a0e131c11f963e2380cd4c582e6bc3
Parents: 27bf4ca
Author: Hequn Cheng <ch...@gmail.com>
Authored: Thu Apr 27 23:03:44 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:55 2017 +0200

----------------------------------------------------------------------
 .../flink/table/api/BatchTableEnvironment.scala |  16 +--
 .../table/api/StreamTableEnvironment.scala      | 122 +++++++++++++++++--
 .../apache/flink/table/api/TableConfig.scala    |  17 +++
 .../flink/table/api/TableEnvironment.scala      |  12 --
 .../datastream/DataStreamRetractionRules.scala  |   5 +-
 .../runtime/CRowInputTupleOutputMapRunner.scala |  63 ++++++++++
 .../apache/flink/table/sinks/CsvTableSink.scala |  98 ++++++++++++++-
 .../flink/table/sinks/StreamRetractSink.scala   |  35 ++++++
 .../flink/table/TableEnvironmentTest.scala      |  62 +---------
 .../scala/batch/TableEnvironmentITCase.scala    |  19 ---
 .../api/scala/stream/RetractionITCase.scala     |   9 +-
 .../api/scala/stream/TableSinkITCase.scala      |  33 ++++-
 .../api/scala/stream/utils/StreamITCase.scala   |   8 --
 13 files changed, 363 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index f7955f0..c7bacfe 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -26,12 +26,11 @@ import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql2rel.RelDecorrelator
 import org.apache.calcite.tools.RuleSet
-import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
+import org.apache.flink.api.common.functions.MapFunction
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.io.DiscardingOutputFormat
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
-import org.apache.flink.configuration.Configuration
 import org.apache.flink.table.explain.PlanJsonParser
 import org.apache.flink.table.expressions.{Expression, RowtimeAttribute, TimeAttribute}
 import org.apache.flink.table.plan.nodes.FlinkConventions
@@ -39,7 +38,6 @@ import org.apache.flink.table.plan.nodes.dataset.DataSetRel
 import org.apache.flink.table.plan.rules.FlinkRuleSets
 import org.apache.flink.table.plan.schema.{DataSetTable, TableSourceTable}
 import org.apache.flink.table.runtime.MapRunner
-import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.table.sinks.{BatchTableSink, TableSink}
 import org.apache.flink.table.sources.{BatchTableSource, TableSource}
 import org.apache.flink.types.Row
@@ -150,18 +148,6 @@ abstract class BatchTableEnvironment(
     if (requestedTypeInfo.getTypeClass == classOf[Row]) {
       // Row to Row, no conversion needed
       None
-    } else if (requestedTypeInfo.getTypeClass == classOf[CRow]) {
-      // Row to CRow, only needs to be wrapped
-      Some(
-        new RichMapFunction[Row, CRow] {
-          private var outCRow: CRow = _
-          override def open(parameters: Configuration): Unit = outCRow = new CRow(null, true)
-          override def map(value: Row): CRow = {
-            outCRow.row = value
-            outCRow
-          }
-        }.asInstanceOf[MapFunction[IN, OUT]]
-      )
     } else {
       // some type that is neither Row or CRow
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index 0632a47..bd06305 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -26,22 +26,25 @@ import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql2rel.RelDecorrelator
 import org.apache.calcite.tools.{RuleSet, RuleSets}
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeinfo.AtomicType
 import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.java.typeutils.GenericTypeInfo
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
+import org.apache.flink.api.java.typeutils.{GenericTypeInfo, TupleTypeInfo}
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.table.calcite.RelTimeIndicatorConverter
 import org.apache.flink.table.explain.PlanJsonParser
 import org.apache.flink.table.expressions.{Expression, ProctimeAttribute, RowtimeAttribute, UnresolvedFieldReference}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.flink.table.plan.nodes.datastream.{DataStreamRel, UpdateAsRetractionTrait}
 import org.apache.flink.table.plan.rules.FlinkRuleSets
-import org.apache.flink.table.plan.schema.{DataStreamTable, StreamTableSourceTable}
-import org.apache.flink.table.runtime.CRowInputMapRunner
+import org.apache.flink.table.plan.schema.StreamTableSourceTable
+import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.table.runtime.{CRowInputMapRunner, CRowInputTupleOutputMapRunner}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
-import org.apache.flink.table.sinks.{StreamTableSink, TableSink}
+import org.apache.flink.table.sinks.{StreamRetractSink, StreamTableSink, TableSink}
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
 import org.apache.flink.table.typeutils.TypeCheckUtils
 import org.apache.flink.types.Row
@@ -130,6 +133,14 @@ abstract class StreamTableEnvironment(
         val result: DataStream[T] = translate(table)(outputType)
         // Give the DataSet to the TableSink to emit it.
         streamSink.emitDataStream(result)
+
+      case streamRetractSink: StreamRetractSink[T] =>
+        val outputType = sink.getOutputType
+        this.config.setNeedsUpdatesAsRetractionForSink(streamRetractSink.needsUpdatesAsRetraction)
+        // translate the Table into a DataStream and provide the type that the TableSink expects.
+        val result: DataStream[JTuple2[Boolean, T]] = translate(table, true)(outputType)
+        // Give the DataSet to the TableSink to emit it.
+        streamRetractSink.emitDataStreamWithChange(result)
       case _ =>
         throw new TableException("StreamTableSink required to emit streaming Table")
     }
@@ -153,7 +164,7 @@ abstract class StreamTableEnvironment(
   Option[MapFunction[IN, OUT]] = {
 
     if (requestedTypeInfo.getTypeClass == classOf[CRow]) {
-      // CRow to CRow, no conversion needed
+      // only used to explain table
       None
     } else if (requestedTypeInfo.getTypeClass == classOf[Row]) {
       // CRow to Row, only needs to be unwrapped
@@ -164,7 +175,6 @@ abstract class StreamTableEnvironment(
       )
     } else {
       // Some type that is neither CRow nor Row
-
       val converterFunction = generateRowConverterFunction[OUT](
         physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
         logicalRowType,
@@ -177,11 +187,40 @@ abstract class StreamTableEnvironment(
         converterFunction.code,
         converterFunction.returnType)
         .asInstanceOf[MapFunction[IN, OUT]])
-
     }
   }
 
   /**
+    * Creates a final converter that maps the internal CRow type to external Tuple2 type.
+    *
+    * @param physicalTypeInfo the input of the sink
+    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
+    * @param requestedTypeInfo the output type of the sink
+    * @param functionName name of the map function. Must not be unique but has to be a
+    *                     valid Java class identifier.
+    */
+  protected def getTupleConversionMapper[IN, OUT](
+      physicalTypeInfo: TypeInformation[IN],
+      logicalRowType: RelDataType,
+      requestedTypeInfo: TypeInformation[OUT],
+      functionName: String):
+  Option[MapFunction[IN, JTuple2[Boolean, OUT]]] = {
+
+    val converterFunction = generateRowConverterFunction(
+      physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
+      logicalRowType,
+      requestedTypeInfo,
+      functionName
+    )
+
+    Some(new CRowInputTupleOutputMapRunner[OUT](
+      converterFunction.name,
+      converterFunction.code,
+      new TupleTypeInfo(BasicTypeInfo.BOOLEAN_TYPE_INFO, requestedTypeInfo))
+           .asInstanceOf[MapFunction[IN, JTuple2[Boolean, OUT]]])
+  }
+
+  /**
     * Registers a [[DataStream]] as a table under a given name in the [[TableEnvironment]]'s
     * catalog.
     *
@@ -371,7 +410,7 @@ abstract class StreamTableEnvironment(
     // 5. optimize the physical Flink plan
     val physicalOptRuleSet = getPhysicalOptRuleSet
     val physicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.DATASTREAM).simplify()
-    val physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
+    var physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
       runVolcanoPlanner(physicalOptRuleSet, logicalPlan, physicalOutputProps)
     } else {
       logicalPlan
@@ -380,6 +419,12 @@ abstract class StreamTableEnvironment(
     // 6. decorate the optimized plan
     val decoRuleSet = getDecoRuleSet
     val decoratedPlan = if (decoRuleSet.iterator().hasNext) {
+
+      if (this.config.getNeedsUpdatesAsRetractionForSink) {
+        physicalPlan = physicalPlan.copy(
+          physicalPlan.getTraitSet.plus(new UpdateAsRetractionTrait(true)),
+          physicalPlan.getInputs)
+      }
       runHepPlanner(HepMatchOrder.BOTTOM_UP, decoRuleSet, physicalPlan, physicalPlan.getTraitSet)
     } else {
       physicalPlan
@@ -388,7 +433,6 @@ abstract class StreamTableEnvironment(
     decoratedPlan
   }
 
-
   /**
     * Translates a [[Table]] into a [[DataStream]].
     *
@@ -444,6 +488,62 @@ abstract class StreamTableEnvironment(
   }
 
   /**
+    * Translates a [[Table]] into a [[DataStream]] with change information.
+    *
+    * The transformation involves optimizing the relational expression tree as defined by
+    * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators.
+    *
+    * @param table       The root node of the relational expression tree.
+    * @param wrapToTuple True, if want to output chang information
+    * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
+    * @tparam A The type of the resulting [[DataStream]].
+    * @return The [[DataStream]] that corresponds to the translated [[Table]].
+    */
+  protected def translate[A](table: Table, wrapToTuple: Boolean)(implicit tpe: TypeInformation[A])
+  : DataStream[JTuple2[Boolean, A]] = {
+    val relNode = table.getRelNode
+    val dataStreamPlan = optimize(relNode)
+    translate(dataStreamPlan, relNode.getRowType, wrapToTuple)
+  }
+
+  /**
+    * Translates a logical [[RelNode]] into a [[DataStream]] with change information.
+    *
+    * @param logicalPlan The root node of the relational expression tree.
+    * @param logicalType The row type of the result. Since the logicalPlan can lose the
+    * @param wrapToTuple True, if want to output chang information
+    * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
+    * @tparam A The type of the resulting [[DataStream]].
+    * @return The [[DataStream]] that corresponds to the translated [[Table]].
+    */
+  protected def translate[A](
+      logicalPlan: RelNode,
+      logicalType: RelDataType,
+      wrapToTuple: Boolean)(implicit tpe: TypeInformation[A]): DataStream[JTuple2[Boolean, A]] = {
+
+    TableEnvironment.validateType(tpe)
+
+    logicalPlan match {
+      case node: DataStreamRel =>
+        val plan = node.translateToPlan(this)
+        val conversion =
+          getTupleConversionMapper(plan.getType, logicalType, tpe, "DataStreamSinkConversion")
+        conversion match {
+          case None => plan.asInstanceOf[DataStream[JTuple2[Boolean, A]]] // no conversion necessary
+          case Some(mapFunction: MapFunction[CRow, JTuple2[Boolean, A]]) =>
+            plan.map(mapFunction)
+              .returns(new TupleTypeInfo[JTuple2[Boolean, A]](BasicTypeInfo.BOOLEAN_TYPE_INFO, tpe))
+              .name(s"to: ${tpe.getTypeClass.getSimpleName}")
+              .asInstanceOf[DataStream[JTuple2[Boolean, A]]]
+        }
+
+      case _ =>
+        throw TableException("Cannot generate DataStream due to an invalid logical plan. " +
+          "This is a bug and should not happen. Please file an issue.")
+    }
+  }
+
+  /**
     * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
     * the result of the given [[Table]].
     *

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
index 6448657..d296978 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
@@ -37,6 +37,11 @@ class TableConfig {
   private var nullCheck: Boolean = true
 
   /**
+    * Defines whether sink table requires that update and delete changes are sent with retraction
+    */
+  private var needsUpdatesAsRetractionForSink: Boolean = false
+
+  /**
     * Defines the configuration of Calcite for Table API and SQL queries.
     */
   private var calciteConfig = CalciteConfig.DEFAULT
@@ -67,6 +72,18 @@ class TableConfig {
   }
 
   /**
+    * Returns the need retraction property for table sink.
+    */
+  def getNeedsUpdatesAsRetractionForSink = needsUpdatesAsRetractionForSink
+
+  /**
+    * Set the need retraction property for table sink.
+    */
+  def setNeedsUpdatesAsRetractionForSink(needsUpdatesAsRetraction: Boolean ): Unit = {
+    this.needsUpdatesAsRetractionForSink = needsUpdatesAsRetraction
+  }
+
+  /**
     * Returns the current configuration of Calcite for Table API and SQL queries.
     */
   def getCalciteConfig: CalciteConfig = calciteConfig

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index d27db1e..5b752ab 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -644,18 +644,6 @@ abstract class TableEnvironment(val config: TableConfig) {
           case _ => throw new TableException(
             "Field reference expression or alias on field expression expected.")
         }
-      case cr: CRowTypeInfo =>
-        exprs.zipWithIndex.map {
-          case (UnresolvedFieldReference(name), idx) => (idx, name)
-          case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
-            val idx = cr.getFieldIndex(origName)
-            if (idx < 0) {
-              throw new TableException(s"$origName is not a field of type $cr")
-            }
-            (idx, name)
-          case _ => throw new TableException(
-            "Field reference expression or alias on field expression expected.")
-        }
       case c: CaseClassTypeInfo[A] =>
         exprs.zipWithIndex flatMap {
           case (UnresolvedFieldReference(name), idx) =>

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
index bd9a7ee..97c0dbb 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
@@ -102,13 +102,14 @@ object DataStreamRetractionRules {
       val rel = call.rel(0).asInstanceOf[DataStreamRel]
       val traits = rel.getTraitSet
 
-      val traitsWithUpdateAsRetrac = if (!traits.contains(UpdateAsRetractionTraitDef.INSTANCE)) {
+      val traitsWithUpdateAsRetrac =
+        if (null == traits.getTrait(UpdateAsRetractionTraitDef.INSTANCE)) {
         traits.plus(UpdateAsRetractionTrait.DEFAULT)
       } else {
         traits
       }
       val traitsWithAccMode =
-        if (!traitsWithUpdateAsRetrac.contains(AccModeTraitDef.INSTANCE)) {
+        if (null == traitsWithUpdateAsRetrac.getTrait(AccModeTraitDef.INSTANCE)) {
           traitsWithUpdateAsRetrac.plus(AccModeTrait.DEFAULT)
       } else {
         traitsWithUpdateAsRetrac

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
new file mode 100644
index 0000000..54bbf7e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala
@@ -0,0 +1,63 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.slf4j.LoggerFactory
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
+
+
+/**
+  * Convert [[CRow]] to a [[Tuple2]]
+  */
+class CRowInputTupleOutputMapRunner[OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[JTuple2[Boolean, OUT]])
+  extends RichMapFunction[CRow, JTuple2[Boolean, OUT]]
+          with ResultTypeQueryable[JTuple2[Boolean, OUT]]
+          with Compiler[MapFunction[Row, OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: MapFunction[Row, OUT] = _
+  private var tupleWrapper: JTuple2[Boolean, OUT] = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating MapFunction.")
+    function = clazz.newInstance()
+    tupleWrapper = new JTuple2[Boolean, OUT]()
+  }
+
+  override def map(in: CRow): JTuple2[Boolean, OUT] = {
+    tupleWrapper.f0 = in.change
+    tupleWrapper.f1 = function.map(in.row)
+    tupleWrapper
+  }
+
+  override def getProducedType: TypeInformation[JTuple2[Boolean, OUT]] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
index 4a2fcdf..809afd2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
@@ -25,7 +25,7 @@ import org.apache.flink.types.Row
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
 
 /**
   * A simple [[TableSink]] to emit data as CSV files.
@@ -135,3 +135,99 @@ class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] {
   }
 }
 
+/**
+  * A simple [[TableSink]] to emit data as CSV files.
+  *
+  * @param path The output path to write the Table to.
+  * @param fieldDelim The field delimiter
+  * @param numFiles The number of files to write to
+  * @param writeMode The write mode to specify whether existing files are overwritten or not.
+  */
+class CsvRetractTableSink(
+    path: String,
+    fieldDelim: Option[String],
+    numFiles: Option[Int],
+    writeMode: Option[WriteMode])
+  extends TableSinkBase[Row] with StreamRetractSink[Row] {
+
+  override def needsUpdatesAsRetraction: Boolean = true
+
+  /**
+    * A simple [[TableSink]] to emit data as CSV files.
+    *
+    * @param path The output path to write the Table to.
+    * @param fieldDelim The field delimiter, ',' by default.
+    */
+  def this(path: String, fieldDelim: String = ",") {
+    this(path, Some(fieldDelim), None, None)
+  }
+
+  /**
+    * A simple [[TableSink]] to emit data as CSV files.
+    *
+    * @param path The output path to write the Table to.
+    * @param fieldDelim The field delimiter.
+    * @param numFiles The number of files to write to.
+    * @param writeMode The write mode to specify whether existing files are overwritten or not.
+    */
+  def this(path: String, fieldDelim: String, numFiles: Int, writeMode: WriteMode) {
+    this(path, Some(fieldDelim), Some(numFiles), Some(writeMode))
+  }
+
+
+  override def emitDataStreamWithChange(dataStream: DataStream[JTuple2[Boolean,Row]]): Unit = {
+    val csvRows = dataStream
+      .map(new CsvRetractFormatter(fieldDelim.getOrElse(",")))
+      .returns(TypeInformation.of(classOf[String]))
+
+
+    if (numFiles.isDefined) {
+      csvRows.setParallelism(numFiles.get)
+    }
+
+    val sink = writeMode match {
+      case None => csvRows.writeAsText(path)
+      case Some(wm) => csvRows.writeAsText(path, wm)
+    }
+
+    if (numFiles.isDefined) {
+      sink.setParallelism(numFiles.get)
+    }
+  }
+
+  override protected def copy: TableSinkBase[Row] = {
+    new CsvRetractTableSink(path, fieldDelim, numFiles, writeMode)
+  }
+
+  override def getOutputType: TypeInformation[Row] = {
+    new RowTypeInfo(getFieldTypes: _*)
+  }
+}
+
+/**
+  * Formats a [[Tuple2]] with change information into a [[String]] with fields separated by the
+  * field delimiter.
+  *
+  * @param fieldDelim The field delimiter.
+  */
+class CsvRetractFormatter(fieldDelim: String) extends MapFunction[JTuple2[Boolean,Row], String] {
+  override def map(rowT: JTuple2[Boolean,Row]): String = {
+
+    val row: Row = rowT.f1
+
+    val builder = new StringBuilder
+
+    builder.append(rowT.f0.toString)
+
+    // write following values
+    for (i <- 0 until row.getArity) {
+      builder.append(fieldDelim)
+      val v = row.getField(i)
+      if (v != null) {
+        builder.append(v.toString)
+      }
+    }
+    builder.mkString
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala
new file mode 100644
index 0000000..7f7c944
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamRetractSink.scala
@@ -0,0 +1,35 @@
+/*
+ * 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.table.sinks
+
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
+import org.apache.flink.streaming.api.datastream.DataStream
+
+trait StreamRetractSink[T] extends TableSink[T]{
+
+  /**
+    * Whether the [[StreamTableSink]] requires that update and delete changes are sent with
+    * retraction messages.
+    */
+  def needsUpdatesAsRetraction: Boolean = false
+
+  /** Emits the DataStream with change infomation. */
+  def emitDataStreamWithChange(dataStream: DataStream[JTuple2[Boolean,T]]): Unit
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
index 60de1f1..675e5d9 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
@@ -21,6 +21,7 @@ package org.apache.flink.table
 import org.apache.flink.api.scala._
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
 import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
 import org.apache.flink.table.api.TableException
@@ -62,14 +63,6 @@ class TableEnvironmentTest extends TableTestBase {
   }
 
   @Test
-  def testGetFieldInfoCRow(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(cRowType)
-
-    fieldInfo._1.zip(Array("f0", "f1", "f2")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
   def testGetFieldInfoCClass(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(caseClassType)
 
@@ -113,20 +106,6 @@ class TableEnvironmentTest extends TableTestBase {
   }
 
   @Test
-  def testGetFieldInfoCRowNames(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      cRowType,
-      Array(
-        UnresolvedFieldReference("name1"),
-        UnresolvedFieldReference("name2"),
-        UnresolvedFieldReference("name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
   def testGetFieldInfoCClassNames(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(
       caseClassType,
@@ -225,45 +204,6 @@ class TableEnvironmentTest extends TableTestBase {
   }
 
   @Test
-  def testGetFieldInfoCRowAlias1(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      cRowType,
-      Array(
-        Alias(UnresolvedFieldReference("f0"), "name1"),
-        Alias(UnresolvedFieldReference("f1"), "name2"),
-        Alias(UnresolvedFieldReference("f2"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoCRowAlias2(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      cRowType,
-      Array(
-        Alias(UnresolvedFieldReference("f2"), "name1"),
-        Alias(UnresolvedFieldReference("f0"), "name2"),
-        Alias(UnresolvedFieldReference("f1"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoCRowAlias3(): Unit = {
-    tEnv.getFieldInfo(
-      cRowType,
-      Array(
-        Alias(UnresolvedFieldReference("xxx"), "name1"),
-        Alias(UnresolvedFieldReference("yyy"), "name2"),
-        Alias(UnresolvedFieldReference("zzz"), "name3")
-      ))
-  }
-
-  @Test
   def testGetFieldInfoCClassAlias1(): Unit = {
     val fieldInfo = tEnv.getFieldInfo(
       caseClassType,

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
index ebfac0a..0c2505a 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
@@ -165,25 +165,6 @@ class TableEnvironmentITCase(
   }
 
   @Test
-  def testToDataSetWithTypeOfCRow(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env)
-      .toTable(tEnv, 'a, 'b, 'c)
-      .select('a, 'b, 'c)
-
-    val expected = "+1,1,Hi\n" + "+2,2,Hello\n" + "+3,2,Hello world\n" +
-      "+4,3,Hello world, how are you?\n" + "+5,3,I am fine.\n" + "+6,3,Luke Skywalker\n" +
-      "+7,4,Comment#1\n" + "+8,4,Comment#2\n" + "+9,4,Comment#3\n" + "+10,4,Comment#4\n" +
-      "+11,5,Comment#5\n" + "+12,5,Comment#6\n" + "+13,5,Comment#7\n" + "+14,5,Comment#8\n" +
-      "+15,5,Comment#9\n" + "+16,6,Comment#10\n" + "+17,6,Comment#11\n" + "+18,6,Comment#12\n" +
-      "+19,6,Comment#13\n" + "+20,6,Comment#14\n" + "+21,6,Comment#15\n"
-    val results = t.toDataSet[CRow].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
   def testToTableFromCaseClass(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env, config)

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
index dde7f89..d490763 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala
@@ -67,13 +67,12 @@ class RetractionITCase extends StreamingWithStateTestBase {
       .groupBy('count)
       .select('count, 'word.count as 'frequency)
 
-    // to DataStream with CRow
-    val results = resultTable.toDataStream[CRow]
-    results.addSink(new StreamITCase.StringSinkWithCRow)
+    val results = resultTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
     env.execute()
 
-    val expected = Seq("+1,1", "+1,2", "+1,1", "+2,1", "+1,2", "+1,1", "+2,2", "+2,1", "+3,1",
-      "+3,0", "+4,1", "+4,0", "+5,1", "+5,0", "+6,1", "+1,2")
+    val expected = Seq("1,1", "1,2", "1,1", "2,1", "1,2", "1,1", "2,2", "2,1", "3,1", "3,0",
+      "4,1", "4,0", "5,1", "5,0", "6,1", "1,2")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
index c446d64..ceae6c6 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
@@ -23,7 +23,7 @@ import java.io.File
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala.stream.utils.StreamTestData
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sinks.CsvTableSink
+import org.apache.flink.table.sinks.{CsvTableSink, CsvRetractTableSink}
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.table.api.TableEnvironment
@@ -59,5 +59,34 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase {
 
     TestBaseUtils.compareResultsByLinesInMemory(expected, path)
   }
-  
+
+  @Test
+  def testStreamTableSinkNeedRetraction(): Unit = {
+
+    val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
+    tmpFile.deleteOnExit()
+    val path = tmpFile.toURI.toString
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setParallelism(4)
+
+    val input = StreamTestData.get3TupleDataStream(env)
+      .map(x => x).setParallelism(1) // increase DOP to 4
+
+    val results = input.toTable(tEnv, 'a, 'b, 'c)
+      .where('a < 5 || 'a > 17)
+      .select('c, 'b)
+      .groupBy('b)
+      .select('b, 'c.count)
+      .writeToSink(new CsvRetractTableSink(path))
+
+    env.execute()
+
+    val expected = Seq(
+      "true,1,1", "true,2,1", "false,2,1", "true,2,2", "true,3,1", "true,6,1", "false,6,1",
+      "true,6,2", "false,6,2", "true,6,3", "false,6,3", "true,6,4").mkString("\n")
+
+    TestBaseUtils.compareResultsByLinesInMemory(expected, path)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bfed279f/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
index 6c75d53..497869d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
@@ -49,12 +49,4 @@ object StreamITCase {
       }
     }
   }
-
-  final class StringSinkWithCRow extends RichSinkFunction[CRow]() {
-    def invoke(value: CRow) {
-      testResults.synchronized {
-        testResults += value.toString
-      }
-    }
-  }
 }


[05/15] flink git commit: [FLINK-5884] [table] Integrate time indicators for Table API & SQL

Posted by fh...@apache.org.
[FLINK-5884] [table] Integrate time indicators for Table API & SQL

This closes #3808.


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

Branch: refs/heads/master
Commit: 495f104b439096dc7eea5141bfe0de0283c5cc62
Parents: 28ab737
Author: twalthr <tw...@apache.org>
Authored: Thu Mar 2 16:06:55 2017 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:31 2017 +0200

----------------------------------------------------------------------
 .../flink/table/api/BatchTableEnvironment.scala |  25 +-
 .../table/api/StreamTableEnvironment.scala      |  79 ++--
 .../flink/table/api/TableEnvironment.scala      | 101 ++++-
 .../flink/table/api/scala/expressionDsl.scala   |  18 +-
 .../apache/flink/table/api/scala/windows.scala  |   2 +-
 .../org/apache/flink/table/api/table.scala      |  14 +-
 .../org/apache/flink/table/api/windows.scala    | 357 +++++++--------
 .../calcite/FlinkCalciteSqlValidator.scala      |  12 +-
 .../flink/table/calcite/FlinkPlannerImpl.scala  |   6 +-
 .../flink/table/calcite/FlinkTypeFactory.scala  | 132 +++++-
 .../calcite/RelTimeIndicatorConverter.scala     | 222 ++++++++++
 .../flink/table/codegen/CodeGenerator.scala     |  99 +++--
 .../table/codegen/calls/FunctionGenerator.scala |  10 -
 .../table/expressions/ExpressionUtils.scala     |  39 ++
 .../apache/flink/table/expressions/call.scala   |  16 +-
 .../table/expressions/fieldExpression.scala     |  48 +-
 .../TimeMaterializationSqlFunction.scala        |  41 ++
 .../functions/TimeModeIndicatorFunctions.scala  | 137 ------
 .../flink/table/plan/ProjectionTranslator.scala |  31 +-
 .../table/plan/logical/LogicalWindow.scala      |  14 +-
 .../flink/table/plan/logical/groupWindows.scala | 280 ++++--------
 .../flink/table/plan/logical/operators.scala    |  40 +-
 .../flink/table/plan/nodes/CommonCalc.scala     |  37 +-
 .../table/plan/nodes/CommonCorrelate.scala      |  32 +-
 .../flink/table/plan/nodes/CommonScan.scala     |  24 +-
 .../flink/table/plan/nodes/OverAggregate.scala  |  35 +-
 .../plan/nodes/PhysicalTableSourceScan.scala    |   6 +-
 .../table/plan/nodes/dataset/BatchScan.scala    |  12 +-
 .../nodes/dataset/BatchTableSourceScan.scala    |  14 +-
 .../plan/nodes/dataset/DataSetAggregate.scala   |   8 +-
 .../table/plan/nodes/dataset/DataSetCalc.scala  |  21 +-
 .../plan/nodes/dataset/DataSetCorrelate.scala   |   9 +-
 .../nodes/dataset/DataSetWindowAggregate.scala  |  68 +--
 .../nodes/datastream/DataStreamAggregate.scala  | 178 ++++----
 .../plan/nodes/datastream/DataStreamCalc.scala  |  22 +-
 .../nodes/datastream/DataStreamCorrelate.scala  |  22 +-
 .../datastream/DataStreamOverAggregate.scala    | 112 ++---
 .../plan/nodes/datastream/DataStreamRel.scala   |   1 -
 .../plan/nodes/datastream/DataStreamScan.scala  |  10 +-
 .../plan/nodes/datastream/DataStreamUnion.scala |  14 +-
 .../nodes/datastream/DataStreamValues.scala     |  21 +-
 .../plan/nodes/datastream/StreamScan.scala      |  32 +-
 .../datastream/StreamTableSourceScan.scala      |  58 ++-
 .../nodes/logical/FlinkLogicalOverWindow.scala  |   2 +-
 .../logical/FlinkLogicalTableSourceScan.scala   |   6 +-
 .../common/WindowStartEndPropertiesRule.scala   |   4 +-
 .../datastream/DataStreamAggregateRule.scala    |   5 +-
 .../rules/datastream/DataStreamCalcRule.scala   |   4 +-
 .../datastream/DataStreamCorrelateRule.scala    |   6 +-
 .../DataStreamLogicalWindowAggregateRule.scala  |  56 ++-
 .../DataStreamOverAggregateRule.scala           |   5 +-
 .../rules/datastream/DataStreamScanRule.scala   |   4 +-
 .../rules/datastream/DataStreamUnionRule.scala  |   3 +-
 .../rules/datastream/DataStreamValuesRule.scala |   3 +-
 .../table/plan/schema/DataStreamTable.scala     |  14 +
 .../flink/table/plan/schema/FlinkTable.scala    |   7 +-
 .../flink/table/plan/schema/RowSchema.scala     | 152 +++++++
 .../plan/schema/TimeIndicatorRelDataType.scala  |  49 +++
 .../apache/flink/table/runtime/MapRunner.scala  |   2 +-
 .../table/runtime/aggregate/AggregateUtil.scala | 190 ++++----
 .../aggregate/RowTimeBoundedRangeOver.scala     |   6 +-
 .../aggregate/RowTimeBoundedRowsOver.scala      |   2 +-
 .../table/sources/DefinedTimeAttributes.scala   |  47 ++
 .../table/typeutils/TimeIndicatorTypeInfo.scala |  45 ++
 .../flink/table/typeutils/TypeCheckUtils.scala  |   5 +-
 .../flink/table/validate/FunctionCatalog.scala  |  10 +-
 .../api/java/batch/TableEnvironmentITCase.java  |   9 -
 .../flink/table/TableEnvironmentTest.scala      |  55 ++-
 .../scala/batch/TableEnvironmentITCase.scala    |  10 -
 .../scala/batch/sql/WindowAggregateTest.scala   |  18 +-
 .../scala/batch/table/FieldProjectionTest.scala |  36 +-
 .../api/scala/batch/table/GroupWindowTest.scala | 121 ++---
 .../table/api/scala/stream/sql/SqlITCase.scala  | 150 ++++---
 .../scala/stream/sql/WindowAggregateTest.scala  | 179 ++++----
 .../scala/stream/table/AggregationsITCase.scala |  12 +-
 .../api/scala/stream/table/CalcITCase.scala     |  16 -
 .../scala/stream/table/GroupWindowTest.scala    | 440 ++++++-------------
 .../scala/stream/table/OverWindowITCase.scala   |  12 +-
 .../api/scala/stream/table/OverWindowTest.scala | 101 ++---
 .../GroupWindowStringExpressionTest.scala       |   6 +-
 .../OverWindowStringExpressionTest.scala        |  16 +-
 .../datastream/DataStreamAggregateITCase.scala  |  22 +-
 82 files changed, 2368 insertions(+), 1921 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index 00cf11c..3eb2ffc 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -194,7 +194,30 @@ abstract class BatchTableEnvironment(
   protected def registerDataSetInternal[T](
       name: String, dataSet: DataSet[T], fields: Array[Expression]): Unit = {
 
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType, fields)
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](
+      dataSet.getType,
+      fields,
+      ignoreTimeAttributes = true)
+
+    // validate and extract time attributes
+    val (rowtime, proctime) = validateAndExtractTimeAttributes(fieldNames, fieldIndexes, fields)
+
+    // don't allow proctime on batch
+    proctime match {
+      case Some(_) =>
+        throw new ValidationException(
+          "A proctime attribute is not allowed in a batch environment. " +
+            "Working with processing-time on batch would lead to non-deterministic results.")
+      case _ => // ok
+    }
+    // rowtime must not extend the schema of a batch table
+    rowtime match {
+      case Some((idx, _)) if idx >= dataSet.getType.getArity =>
+        throw new ValidationException(
+          "A rowtime attribute must be defined on an existing field in a batch environment.")
+      case _ => // ok
+    }
+
     val dataSetTable = new DataSetTable[T](dataSet, fieldIndexes, fieldNames)
     registerTableInternal(name, dataSetTable)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index f532c5b..d1f2fb5 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.table.calcite.RelTimeIndicatorConverter
 import org.apache.flink.table.explain.PlanJsonParser
 import org.apache.flink.table.expressions.Expression
 import org.apache.flink.table.plan.nodes.FlinkConventions
@@ -87,47 +88,6 @@ abstract class StreamTableEnvironment(
   protected def createUniqueTableName(): String = "_DataStreamTable_" + nameCntr.getAndIncrement()
 
   /**
-    * Returns field names and field positions for a given [[TypeInformation]].
-    *
-    * Field names are automatically extracted for
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    * The method fails if inputType is not a
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    *
-    * @param inputType The TypeInformation extract the field names and positions from.
-    * @tparam A The type of the TypeInformation.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  override protected[flink] def getFieldInfo[A](inputType: TypeInformation[A])
-    : (Array[String], Array[Int]) = {
-    val fieldInfo = super.getFieldInfo(inputType)
-    if (fieldInfo._1.contains("rowtime")) {
-      throw new TableException("'rowtime' ia a reserved field name in stream environment.")
-    }
-    fieldInfo
-  }
-
-  /**
-    * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
-    * [[Expression]].
-    *
-    * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
-    * @param exprs     The expressions that define the field names.
-    * @tparam A The type of the TypeInformation.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  override protected[flink] def getFieldInfo[A](
-      inputType: TypeInformation[A],
-      exprs: Array[Expression])
-    : (Array[String], Array[Int]) = {
-    val fieldInfo = super.getFieldInfo(inputType, exprs)
-    if (fieldInfo._1.contains("rowtime")) {
-      throw new TableException("'rowtime' is a reserved field name in stream environment.")
-    }
-    fieldInfo
-  }
-
-  /**
     * Registers an external [[StreamTableSource]] in this [[TableEnvironment]]'s catalog.
     * Registered tables can be referenced in SQL queries.
     *
@@ -145,6 +105,7 @@ abstract class StreamTableEnvironment(
             "StreamTableEnvironment")
     }
   }
+
   /**
     * Writes a [[Table]] to a [[TableSink]].
     *
@@ -185,7 +146,9 @@ abstract class StreamTableEnvironment(
     val dataStreamTable = new DataStreamTable[T](
       dataStream,
       fieldIndexes,
-      fieldNames
+      fieldNames,
+      None,
+      None
     )
     registerTableInternal(name, dataStreamTable)
   }
@@ -200,15 +163,26 @@ abstract class StreamTableEnvironment(
     * @tparam T The type of the [[DataStream]].
     */
   protected def registerDataStreamInternal[T](
-    name: String,
-    dataStream: DataStream[T],
-    fields: Array[Expression]): Unit = {
+      name: String,
+      dataStream: DataStream[T],
+      fields: Array[Expression])
+    : Unit = {
+
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](
+      dataStream.getType,
+      fields,
+      ignoreTimeAttributes = false)
+
+    // validate and extract time attributes
+    val (rowtime, proctime) = validateAndExtractTimeAttributes(fieldNames, fieldIndexes, fields)
+
 
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataStream.getType, fields)
     val dataStreamTable = new DataStreamTable[T](
       dataStream,
       fieldIndexes,
-      fieldNames
+      fieldNames,
+      rowtime,
+      proctime
     )
     registerTableInternal(name, dataStreamTable)
   }
@@ -259,7 +233,10 @@ abstract class StreamTableEnvironment(
     // 1. decorrelate
     val decorPlan = RelDecorrelator.decorrelateQuery(relNode)
 
-    // 2. normalize the logical plan
+    // 2. convert time indicators
+    val convPlan = RelTimeIndicatorConverter.convert(decorPlan, getRelBuilder.getRexBuilder)
+
+    // 3. normalize the logical plan
     val normRuleSet = getNormRuleSet
     val normalizedPlan = if (normRuleSet.iterator().hasNext) {
       runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, decorPlan, decorPlan.getTraitSet)
@@ -267,7 +244,7 @@ abstract class StreamTableEnvironment(
       decorPlan
     }
 
-    // 3. optimize the logical Flink plan
+    // 4. optimize the logical Flink plan
     val logicalOptRuleSet = getLogicalOptRuleSet
     val logicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.LOGICAL).simplify()
     val logicalPlan = if (logicalOptRuleSet.iterator().hasNext) {
@@ -276,7 +253,7 @@ abstract class StreamTableEnvironment(
       normalizedPlan
     }
 
-    // 4. optimize the physical Flink plan
+    // 5. optimize the physical Flink plan
     val physicalOptRuleSet = getPhysicalOptRuleSet
     val physicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.DATASTREAM).simplify()
     val physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
@@ -285,7 +262,7 @@ abstract class StreamTableEnvironment(
       logicalPlan
     }
 
-    // 5. decorate the optimized plan
+    // 6. decorate the optimized plan
     val decoRuleSet = getDecoRuleSet
     val decoratedPlan = if (decoRuleSet.iterator().hasNext) {
       runHepPlanner(HepMatchOrder.BOTTOM_UP, decoRuleSet, physicalPlan, physicalPlan.getTraitSet)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 06c405e..4c72e8f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -52,6 +52,9 @@ import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
 import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{ScalarFunction, TableFunction, AggregateFunction}
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions}
+import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
 import org.apache.flink.table.plan.cost.DataSetCostFactory
 import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode}
 import org.apache.flink.table.plan.rules.FlinkRuleSets
@@ -598,70 +601,94 @@ abstract class TableEnvironment(val config: TableConfig) {
 
   /**
     * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
-    * [[Expression]].
+    * [[Expression]]. It does not handle time attributes but considers them in indices, if
+    * ignore flag is not false.
     *
     * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
     * @param exprs The expressions that define the field names.
+    * @param ignoreTimeAttributes ignore time attributes and handle them as regular expressions.
     * @tparam A The type of the TypeInformation.
     * @return A tuple of two arrays holding the field names and corresponding field positions.
     */
   protected[flink] def getFieldInfo[A](
-    inputType: TypeInformation[A],
-    exprs: Array[Expression]): (Array[String], Array[Int]) = {
+      inputType: TypeInformation[A],
+      exprs: Array[Expression],
+      ignoreTimeAttributes: Boolean)
+    : (Array[String], Array[Int]) = {
 
     TableEnvironment.validateType(inputType)
 
+    val filteredExprs = if (ignoreTimeAttributes) {
+        exprs.map {
+          case ta: TimeAttribute => ta.expression
+          case e@_ => e
+        }
+    } else {
+      exprs
+    }
+
     val indexedNames: Array[(Int, String)] = inputType match {
       case g: GenericTypeInfo[A] if g.getTypeClass == classOf[Row] =>
         throw new TableException(
           "An input of GenericTypeInfo<Row> cannot be converted to Table. " +
             "Please specify the type of the input with a RowTypeInfo.")
       case a: AtomicType[A] =>
-        if (exprs.length != 1) {
-          throw new TableException("Table of atomic type can only have a single field.")
-        }
-        exprs.map {
-          case UnresolvedFieldReference(name) => (0, name)
+        filteredExprs.zipWithIndex flatMap {
+          case (UnresolvedFieldReference(name), idx) =>
+            if (idx > 0) {
+              throw new TableException("Table of atomic type can only have a single field.")
+            }
+            Some((0, name))
+          case (_: TimeAttribute, _) if ignoreTimeAttributes =>
+            None
           case _ => throw new TableException("Field reference expression requested.")
         }
       case t: TupleTypeInfo[A] =>
-        exprs.zipWithIndex.map {
-          case (UnresolvedFieldReference(name), idx) => (idx, name)
+        filteredExprs.zipWithIndex flatMap {
+          case (UnresolvedFieldReference(name), idx) =>
+            Some((idx, name))
           case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
             val idx = t.getFieldIndex(origName)
             if (idx < 0) {
               throw new TableException(s"$origName is not a field of type $t")
             }
-            (idx, name)
+            Some((idx, name))
+          case (_: TimeAttribute, _) =>
+            None
           case _ => throw new TableException(
             "Field reference expression or alias on field expression expected.")
         }
       case c: CaseClassTypeInfo[A] =>
-        exprs.zipWithIndex.map {
-          case (UnresolvedFieldReference(name), idx) => (idx, name)
+        filteredExprs.zipWithIndex flatMap {
+          case (UnresolvedFieldReference(name), idx) =>
+            Some((idx, name))
           case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
             val idx = c.getFieldIndex(origName)
             if (idx < 0) {
               throw new TableException(s"$origName is not a field of type $c")
             }
-            (idx, name)
+            Some((idx, name))
+          case (_: TimeAttribute, _) =>
+            None
           case _ => throw new TableException(
             "Field reference expression or alias on field expression expected.")
         }
       case p: PojoTypeInfo[A] =>
-        exprs.map {
+        filteredExprs flatMap {
           case (UnresolvedFieldReference(name)) =>
             val idx = p.getFieldIndex(name)
             if (idx < 0) {
               throw new TableException(s"$name is not a field of type $p")
             }
-            (idx, name)
+            Some((idx, name))
           case Alias(UnresolvedFieldReference(origName), name, _) =>
             val idx = p.getFieldIndex(origName)
             if (idx < 0) {
               throw new TableException(s"$origName is not a field of type $p")
             }
-            (idx, name)
+            Some((idx, name))
+          case _: TimeAttribute =>
+            None
           case _ => throw new TableException(
             "Field reference expression or alias on field expression expected.")
         }
@@ -795,6 +822,42 @@ abstract class TableEnvironment(val config: TableConfig) {
     Some(mapFunction)
   }
 
+  /**
+    * Checks for at most one rowtime and proctime attribute.
+    * Returns the time attributes.
+    *
+    * @return rowtime attribute and proctime attribute
+    */
+  protected def validateAndExtractTimeAttributes(
+      fieldNames: Seq[String],
+      fieldIndices: Seq[Int],
+      exprs: Array[Expression])
+    : (Option[(Int, String)], Option[(Int, String)]) = {
+
+    var rowtime: Option[(Int, String)] = None
+    var proctime: Option[(Int, String)] = None
+
+    exprs.zipWithIndex.foreach {
+      case (RowtimeAttribute(reference@UnresolvedFieldReference(name)), idx) =>
+        if (rowtime.isDefined) {
+          throw new TableException(
+            "The rowtime attribute can only be defined once in a table schema.")
+        } else {
+          rowtime = Some(idx, name)
+        }
+      case (ProctimeAttribute(reference@UnresolvedFieldReference(name)), idx) =>
+        if (proctime.isDefined) {
+          throw new TableException(
+            "The proctime attribute can only be defined once in a table schema.")
+        } else {
+          proctime = Some(idx, name)
+        }
+      case _ =>
+        // do nothing
+    }
+
+    (rowtime, proctime)
+  }
 }
 
 /**
@@ -803,6 +866,10 @@ abstract class TableEnvironment(val config: TableConfig) {
   */
 object TableEnvironment {
 
+  // default names that can be used in in TableSources etc.
+  val DEFAULT_ROWTIME_ATTRIBUTE = "rowtime"
+  val DEFAULT_PROCTIME_ATTRIBUTE = "proctime"
+
   /**
     * Returns a [[JavaBatchTableEnv]] for a Java [[JavaBatchExecEnv]].
     *

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
index cc58ff5..6d15212 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
@@ -625,7 +625,7 @@ trait ImplicitExpressionOperations {
     */
   def millis = milli
 
-  // row interval type
+  // Row interval type
 
   /**
     * Creates an interval of rows.
@@ -634,6 +634,8 @@ trait ImplicitExpressionOperations {
     */
   def rows = toRowInterval(expr)
 
+  // Advanced type helper functions
+
   /**
     * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and
     * returns it's value.
@@ -680,6 +682,20 @@ trait ImplicitExpressionOperations {
     * @return the first and only element of an array with a single element
     */
   def element() = ArrayElement(expr)
+
+  // Schema definition
+
+  /**
+    * Declares a field as the rowtime attribute for indicating, accessing, and working in
+    * Flink's event time.
+    */
+  def rowtime = RowtimeAttribute(expr)
+
+  /**
+    * Declares a field as the proctime attribute for indicating, accessing, and working in
+    * Flink's processing time.
+    */
+  def proctime = ProctimeAttribute(expr)
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/windows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/windows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/windows.scala
index 5e70440..d0430c2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/windows.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/windows.scala
@@ -18,8 +18,8 @@
 
 package org.apache.flink.table.api.scala
 
+import org.apache.flink.table.api.{OverWindowWithOrderBy, SessionWithGap, SlideWithSize, TumbleWithSize}
 import org.apache.flink.table.expressions.Expression
-import org.apache.flink.table.api.{TumbleWithSize, OverWindowWithOrderBy, SlideWithSize, SessionWithGap}
 
 /**
   * Helper object for creating a tumbling window. Tumbling windows are consecutive, non-overlapping

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
index 87dde0a..dd8265b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
@@ -20,12 +20,12 @@ package org.apache.flink.table.api
 import org.apache.calcite.rel.RelNode
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.operators.join.JoinType
-import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
+import org.apache.flink.table.calcite.FlinkRelBuilder
+import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.expressions.{Alias, Asc, Expression, ExpressionParser, Ordering, UnresolvedAlias, UnresolvedFieldReference}
-import org.apache.flink.table.plan.logical.Minus
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.plan.ProjectionTranslator._
-import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.logical.{Minus, _}
 import org.apache.flink.table.sinks.TableSink
 
 import _root_.scala.collection.JavaConverters._
@@ -1015,13 +1015,7 @@ class WindowGroupedTable(
     val projectsOnAgg = replaceAggregationsAndProperties(
       fields, table.tableEnv, aggNames, propNames)
 
-    val projectFields = (table.tableEnv, window) match {
-      // event time can be arbitrary field in batch environment
-      case (_: BatchTableEnvironment, w: EventTimeWindow) =>
-        extractFieldReferences(fields ++ groupKeys ++ Seq(w.timeField))
-      case (_, _) =>
-        extractFieldReferences(fields ++ groupKeys)
-    }
+    val projectFields = extractFieldReferences(fields ++ groupKeys :+ window.timeField)
 
     new Table(table.tableEnv,
       Project(

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
index 80260f7..11ef360 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
@@ -149,7 +149,7 @@ class OverWindowWithOrderBy(
   * A window specification.
   *
   * Window groups rows based on time or row-count intervals. It is a general way to group the
-  * elements, which is very helpful for both groupby-aggregations and over-aggregations to
+  * elements, which is very helpful for both groupBy-aggregations and over-aggregations to
   * compute aggregates on groups of elements.
   *
   * Infinite streaming tables can only be grouped into time or row intervals. Hence window grouping
@@ -157,111 +157,73 @@ class OverWindowWithOrderBy(
   *
   * For finite batch tables, window provides shortcuts for time-based groupBy.
   *
-  * @param alias The expression of alias for this Window
   */
-abstract class Window(val alias: Expression) {
+abstract class Window(val alias: Expression, val timeField: Expression) {
 
   /**
     * Converts an API class to a logical window for planning.
     */
   private[flink] def toLogicalWindow: LogicalWindow
+
 }
 
+// ------------------------------------------------------------------------------------------------
+// Tumbling windows
+// ------------------------------------------------------------------------------------------------
+
 /**
-  * A window specification without alias.
+  * Tumbling window.
+  *
+  * For streaming tables you can specify grouping by a event-time or processing-time attribute.
+  *
+  * For batch tables you can specify grouping on a timestamp or long attribute.
+  *
+  * @param size the size of the window either as time or row-count interval.
   */
-abstract class WindowWithoutAlias {
+class TumbleWithSize(size: Expression) {
 
   /**
-    * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
-    * refer to. `select()` statement can access window properties such as window start or end time.
+    * Tumbling window.
     *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: Expression): Window
-
-  /**
-    * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
-    * refer to. `select()` statement can access window properties such as window start or end time.
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
     *
-    * @param alias alias for this window
-    * @return this window
+    * For batch tables you can specify grouping on a timestamp or long attribute.
+    *
+    * @param size the size of the window either as time or row-count interval.
     */
-  def as(alias: String): Window = as(ExpressionParser.parseExpression(alias))
-}
-
-/**
-  * A predefined specification of window on processing-time
-  */
-abstract class ProcTimeWindowWithoutAlias extends WindowWithoutAlias {
+  def this(size: String) = this(ExpressionParser.parseExpression(size))
 
   /**
     * Specifies the time attribute on which rows are grouped.
     *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
     *
-    * For batch tables, refer to a timestamp or long attribute.
+    * For batch tables you can specify grouping on a timestamp or long attribute.
     *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a predefined window on event-time
+    * @param timeField time attribute for streaming and batch tables
+    * @return a tumbling window on event-time
     */
-  def on(timeField: Expression): WindowWithoutAlias
+  def on(timeField: Expression): TumbleWithSizeOnTime =
+    new TumbleWithSizeOnTime(timeField, size)
 
   /**
     * Specifies the time attribute on which rows are grouped.
     *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
     *
-    * For batch tables, refer to a timestamp or long attribute.
+    * For batch tables you can specify grouping on a timestamp or long attribute.
     *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a predefined window on event-time
+    * @param timeField time attribute for streaming and batch tables
+    * @return a tumbling window on event-time
     */
-  def on(timeField: String): WindowWithoutAlias =
+  def on(timeField: String): TumbleWithSizeOnTime =
     on(ExpressionParser.parseExpression(timeField))
 }
 
 /**
-  * A window operating on event-time.
-  *
-  * For streaming tables call on('rowtime) to specify grouping by event-time.
-  * Otherwise rows are grouped by processing-time.
-  *
-  * For batch tables, refer to a timestamp or long attribute.
-  *
-  * @param timeField time mode for streaming tables and time attribute for batch tables
+  * Tumbling window on time.
   */
-abstract class EventTimeWindow(alias: Expression, val timeField: Expression) extends Window(alias)
-
-// ------------------------------------------------------------------------------------------------
-// Tumbling windows
-// ------------------------------------------------------------------------------------------------
-
-/**
-  * A partial specification of a tumbling window.
-  *
-  * @param size the size of the window either a time or a row-count interval.
-  */
-class TumbleWithSize(size: Expression) extends ProcTimeWindowWithoutAlias {
-
-  def this(size: String) = this(ExpressionParser.parseExpression(size))
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time.
-    * Otherwise rows are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a predefined window on event-time
-    */
-  override def on(timeField: Expression): WindowWithoutAlias =
-    new TumbleWithoutAlias(timeField, size)
+class TumbleWithSizeOnTime(time: Expression, size: Expression) {
 
   /**
     * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
@@ -270,15 +232,9 @@ class TumbleWithSize(size: Expression) extends ProcTimeWindowWithoutAlias {
     * @param alias alias for this window
     * @return this window
     */
-  override def as(alias: Expression) = new TumblingWindow(alias, size)
-}
-
-/**
-  * A tumbling window on event-time without alias.
-  */
-class TumbleWithoutAlias(
-    time: Expression,
-    size: Expression) extends WindowWithoutAlias {
+  def as(alias: Expression): TumbleWithSizeOnTimeWithAlias = {
+    new TumbleWithSizeOnTimeWithAlias(alias, time, size)
+  }
 
   /**
     * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
@@ -287,31 +243,28 @@ class TumbleWithoutAlias(
     * @param alias alias for this window
     * @return this window
     */
-  override def as(alias: Expression): Window = new TumblingEventTimeWindow(alias, time, size)
-}
-
-/**
-  * Tumbling window on processing-time.
-  *
-  * @param alias the alias of the window.
-  * @param size the size of the window either a time or a row-count interval.
-  */
-class TumblingWindow(alias: Expression, size: Expression) extends Window(alias) {
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    ProcessingTimeTumblingGroupWindow(alias, size)
+  def as(alias: String): TumbleWithSizeOnTimeWithAlias = {
+    as(ExpressionParser.parseExpression(alias))
+  }
 }
 
 /**
-  * Tumbling window on event-time.
+  * Tumbling window on time with alias. Fully specifies a window.
   */
-class TumblingEventTimeWindow(
+class TumbleWithSizeOnTimeWithAlias(
     alias: Expression,
-    time: Expression,
-    size: Expression) extends EventTimeWindow(alias, time) {
+    timeField: Expression,
+    size: Expression)
+  extends Window(
+    alias,
+    timeField) {
 
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    EventTimeTumblingGroupWindow(alias, time, size)
+  /**
+    * Converts an API class to a logical window for planning.
+    */
+  override private[flink] def toLogicalWindow: LogicalWindow = {
+    TumblingGroupWindow(alias, timeField, size)
+  }
 }
 
 // ------------------------------------------------------------------------------------------------
@@ -319,16 +272,16 @@ class TumblingEventTimeWindow(
 // ------------------------------------------------------------------------------------------------
 
 /**
-  * A partially specified sliding window.
+  * Partially specified sliding window.
   *
-  * @param size the size of the window either a time or a row-count interval.
+  * @param size the size of the window either as time or row-count interval.
   */
 class SlideWithSize(size: Expression) {
 
   /**
-    * A partially specified sliding window.
+    * Partially specified sliding window.
     *
-    * @param size the size of the window either a time or a row-count interval.
+    * @param size the size of the window either as time or row-count interval.
     */
   def this(size: String) = this(ExpressionParser.parseExpression(size))
 
@@ -343,9 +296,9 @@ class SlideWithSize(size: Expression) {
     * windows.
     *
     * @param slide the slide of the window either as time or row-count interval.
-    * @return a predefined sliding window.
+    * @return a sliding window
     */
-  def every(slide: Expression): SlideWithSlide = new SlideWithSlide(size, slide)
+  def every(slide: Expression): SlideWithSizeAndSlide = new SlideWithSizeAndSlide(size, slide)
 
   /**
     * Specifies the window's slide as time or row-count interval.
@@ -358,48 +311,54 @@ class SlideWithSize(size: Expression) {
     * windows.
     *
     * @param slide the slide of the window either as time or row-count interval.
-    * @return a predefined sliding window.
+    * @return a sliding window
     */
-  def every(slide: String): WindowWithoutAlias = every(ExpressionParser.parseExpression(slide))
+  def every(slide: String): SlideWithSizeAndSlide = every(ExpressionParser.parseExpression(slide))
 }
 
 /**
-  * A partially defined sliding window.
+  * Sliding window.
+  *
+  * For streaming tables you can specify grouping by a event-time or processing-time attribute.
+  *
+  * For batch tables you can specify grouping on a timestamp or long attribute.
+  *
+  * @param size the size of the window either as time or row-count interval.
   */
-class SlideWithSlide(
-    size: Expression,
-    slide: Expression) extends ProcTimeWindowWithoutAlias {
+class SlideWithSizeAndSlide(size: Expression, slide: Expression) {
+
   /**
     * Specifies the time attribute on which rows are grouped.
     *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
     *
-    * For batch tables, refer to a timestamp or long attribute.
+    * For batch tables you can specify grouping on a timestamp or long attribute.
     *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a predefined Sliding window on event-time.
+    * @param timeField time attribute for streaming and batch tables
+    * @return a tumbling window on event-time
     */
-  override def on(timeField: Expression): SlideWithoutAlias =
-    new SlideWithoutAlias(timeField, size, slide)
+  def on(timeField: Expression): SlideWithSizeAndSlideOnTime =
+    new SlideWithSizeAndSlideOnTime(timeField, size, slide)
 
   /**
-    * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
-    * refer to. `select()` statement can access window properties such as window start or end time.
+    * Specifies the time attribute on which rows are grouped.
     *
-    * @param alias alias for this window
-    * @return this window
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
+    *
+    * For batch tables you can specify grouping on a timestamp or long attribute.
+    *
+    * @param timeField time attribute for streaming and batch tables
+    * @return a tumbling window on event-time
     */
-  override def as(alias: Expression): Window = new SlidingWindow(alias, size, slide)
+  def on(timeField: String): SlideWithSizeAndSlideOnTime =
+    on(ExpressionParser.parseExpression(timeField))
 }
 
 /**
-  * A partially defined sliding window on event-time without alias.
+  * Sliding window on time.
   */
-class SlideWithoutAlias(
-    timeField: Expression,
-    size: Expression,
-    slide: Expression) extends WindowWithoutAlias {
+class SlideWithSizeAndSlideOnTime(timeField: Expression, size: Expression, slide: Expression) {
+
   /**
     * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
     * refer to. `select()` statement can access window properties such as window start or end time.
@@ -407,39 +366,40 @@ class SlideWithoutAlias(
     * @param alias alias for this window
     * @return this window
     */
-  override def as(alias: Expression): Window =
-    new SlidingEventTimeWindow(alias, timeField, size, slide)
-}
+  def as(alias: Expression): SlideWithSizeAndSlideOnTimeWithAlias = {
+    new SlideWithSizeAndSlideOnTimeWithAlias(alias, timeField, size, slide)
+  }
 
-/**
-  * A sliding window on processing-time.
-  *
-  * @param alias the alias of the window.
-  * @param size the size of the window either a time or a row-count interval.
-  * @param slide the interval by which the window slides.
-  */
-class SlidingWindow(
-  alias: Expression,
-  size: Expression,
-  slide: Expression)
-  extends Window(alias) {
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    ProcessingTimeSlidingGroupWindow(alias, size, slide)
+  /**
+    * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
+    * refer to. `select()` statement can access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: String): SlideWithSizeAndSlideOnTimeWithAlias = {
+    as(ExpressionParser.parseExpression(alias))
+  }
 }
 
 /**
-  * A sliding window on event-time.
+  * Sliding window on time with alias. Fully specifies a window.
   */
-class SlidingEventTimeWindow(
+class SlideWithSizeAndSlideOnTimeWithAlias(
     alias: Expression,
     timeField: Expression,
     size: Expression,
     slide: Expression)
-  extends EventTimeWindow(alias, timeField) {
+  extends Window(
+    alias,
+    timeField) {
 
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    EventTimeSlidingGroupWindow(alias, timeField, size, slide)
+  /**
+    * Converts an API class to a logical window for planning.
+    */
+  override private[flink] def toLogicalWindow: LogicalWindow = {
+    SlidingGroupWindow(alias, timeField, size, slide)
+  }
 }
 
 // ------------------------------------------------------------------------------------------------
@@ -447,42 +407,59 @@ class SlidingEventTimeWindow(
 // ------------------------------------------------------------------------------------------------
 
 /**
-  * A partially defined session window.
+  * Session window.
+  *
+  * For streaming tables you can specify grouping by a event-time or processing-time attribute.
+  *
+  * For batch tables you can specify grouping on a timestamp or long attribute.
+  *
+  * @param gap the time interval of inactivity before a window is closed.
   */
-class SessionWithGap(gap: Expression) extends ProcTimeWindowWithoutAlias {
+class SessionWithGap(gap: Expression) {
 
+  /**
+    * Session window.
+    *
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
+    *
+    * For batch tables you can specify grouping on a timestamp or long attribute.
+    *
+    * @param gap the time interval of inactivity before a window is closed.
+    */
   def this(gap: String) = this(ExpressionParser.parseExpression(gap))
 
   /**
     * Specifies the time attribute on which rows are grouped.
     *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
     *
-    * For batch tables, refer to a timestamp or long attribute.
+    * For batch tables you can specify grouping on a timestamp or long attribute.
     *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return an on event-time session window on event-time
+    * @param timeField time attribute for streaming and batch tables
+    * @return a tumbling window on event-time
     */
-  override def on(timeField: Expression): SessionWithoutAlias =
-    new SessionWithoutAlias(timeField, gap)
+  def on(timeField: Expression): SessionWithGapOnTime =
+    new SessionWithGapOnTime(timeField, gap)
 
   /**
-    * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
-    * refer to. `select()` statement can access window properties such as window start or end time.
+    * Specifies the time attribute on which rows are grouped.
     *
-    * @param alias alias for this window
-    * @return this window
+    * For streaming tables you can specify grouping by a event-time or processing-time attribute.
+    *
+    * For batch tables you can specify grouping on a timestamp or long attribute.
+    *
+    * @param timeField time attribute for streaming and batch tables
+    * @return a tumbling window on event-time
     */
-  override def as(alias: Expression): Window = new SessionWindow(alias, gap)
+  def on(timeField: String): SessionWithGapOnTime =
+    on(ExpressionParser.parseExpression(timeField))
 }
 
 /**
-  * A partially defined session window on event-time without alias.
+  * Session window on time.
   */
-class SessionWithoutAlias(
-    timeField: Expression,
-    gap: Expression) extends WindowWithoutAlias {
+class SessionWithGapOnTime(timeField: Expression, gap: Expression) {
+
   /**
     * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
     * refer to. `select()` statement can access window properties such as window start or end time.
@@ -490,29 +467,37 @@ class SessionWithoutAlias(
     * @param alias alias for this window
     * @return this window
     */
-  override def as(alias: Expression): Window = new SessionEventTimeWindow(alias, timeField, gap)
-}
-
-/**
-  * A session window on processing-time.
-  *
-  * @param gap the time interval of inactivity before a window is closed.
-  */
-class SessionWindow(alias: Expression, gap: Expression) extends Window(alias) {
+  def as(alias: Expression): SessionWithGapOnTimeWithAlias = {
+    new SessionWithGapOnTimeWithAlias(alias, timeField, gap)
+  }
 
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    ProcessingTimeSessionGroupWindow(alias, gap)
+  /**
+    * Assigns an alias for this window that the following `groupBy()` and `select()` clause can
+    * refer to. `select()` statement can access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: String): SessionWithGapOnTimeWithAlias = {
+    as(ExpressionParser.parseExpression(alias))
+  }
 }
 
 /**
-  * A session window on event-time.
+  * Session window on time with alias. Fully specifies a window.
   */
-class SessionEventTimeWindow(
+class SessionWithGapOnTimeWithAlias(
     alias: Expression,
     timeField: Expression,
     gap: Expression)
-  extends EventTimeWindow(alias, timeField) {
+  extends Window(
+    alias,
+    timeField) {
 
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    EventTimeSessionGroupWindow(alias, timeField, gap)
+  /**
+    * Converts an API class to a logical window for planning.
+    */
+  override private[flink] def toLogicalWindow: LogicalWindow = {
+    SessionGroupWindow(alias, timeField, gap)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
index b4a3c42..2bdf360 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
@@ -21,8 +21,8 @@ package org.apache.flink.table.calcite
 import org.apache.calcite.adapter.java.JavaTypeFactory
 import org.apache.calcite.prepare.CalciteCatalogReader
 import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql.validate.{SqlConformance, SqlValidatorImpl}
-import org.apache.calcite.sql.{SqlInsert, SqlOperatorTable}
+import org.apache.calcite.sql._
+import org.apache.calcite.sql.validate.{SqlConformanceEnum, SqlValidatorImpl}
 
 /**
  * This is a copy of Calcite's CalciteSqlValidator to use with [[FlinkPlannerImpl]].
@@ -30,8 +30,12 @@ import org.apache.calcite.sql.{SqlInsert, SqlOperatorTable}
 class FlinkCalciteSqlValidator(
     opTab: SqlOperatorTable,
     catalogReader: CalciteCatalogReader,
-    typeFactory: JavaTypeFactory) extends SqlValidatorImpl(
-        opTab, catalogReader, typeFactory, SqlConformance.DEFAULT) {
+    factory: JavaTypeFactory)
+  extends SqlValidatorImpl(
+    opTab,
+    catalogReader,
+    factory,
+    SqlConformanceEnum.DEFAULT) {
 
   override def getLogicalSourceRowType(
       sourceRowType: RelDataType,

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
index 09e3277..beb2436 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
@@ -107,7 +107,11 @@ class FlinkPlannerImpl(
       // we disable automatic flattening in order to let composite types pass without modification
       // we might enable it again once Calcite has better support for structured types
       // root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true))
-      root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel))
+
+      // TableEnvironment.optimize will execute the following
+      // root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel))
+      // convert time indicators
+      // root = root.withRel(RelTimeIndicatorConverter.convert(root.rel, rexBuilder))
       root
     } catch {
       case e: RelConversionException => throw TableException(e.getMessage)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
index 7762ff8..001011b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
@@ -20,25 +20,25 @@ package org.apache.flink.table.calcite
 
 import org.apache.calcite.avatica.util.TimeUnit
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeSystem}
+import org.apache.calcite.rel.`type`._
 import org.apache.calcite.sql.SqlIntervalQualifier
-import org.apache.calcite.sql.`type`.SqlTypeName
 import org.apache.calcite.sql.`type`.SqlTypeName._
+import org.apache.calcite.sql.`type`.{BasicSqlType, SqlTypeName}
 import org.apache.calcite.sql.parser.SqlParserPos
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
 import org.apache.flink.api.common.typeinfo.{NothingTypeInfo, PrimitiveArrayTypeInfo, SqlTimeTypeInfo, TypeInformation}
 import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo}
 import org.apache.flink.api.java.typeutils.ValueTypeInfo._
+import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo}
 import org.apache.flink.table.api.TableException
-import org.apache.flink.table.plan.schema.{ArrayRelDataType, CompositeRelDataType, GenericRelDataType, MapRelDataType}
-import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
-import org.apache.flink.table.typeutils.TypeCheckUtils.isSimple
 import org.apache.flink.table.calcite.FlinkTypeFactory.typeInfoToSqlTypeName
+import org.apache.flink.table.plan.schema._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isSimple
+import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TimeIntervalTypeInfo}
 import org.apache.flink.types.Row
 
-import scala.collection.mutable
 import scala.collection.JavaConverters._
+import scala.collection.mutable
 
 /**
   * Flink specific type factory that represents the interface between Flink's [[TypeInformation]]
@@ -65,6 +65,12 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp
           createSqlIntervalType(
             new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO))
 
+        case TimeIndicatorTypeInfo.ROWTIME_INDICATOR =>
+          createRowtimeIndicatorType()
+
+        case TimeIndicatorTypeInfo.PROCTIME_INDICATOR =>
+          createProctimeIndicatorType()
+
         case _ =>
           createSqlType(sqlType)
       }
@@ -77,23 +83,75 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp
   }
 
   /**
+    * Creates a indicator type for processing-time, but with similar properties as SQL timestamp.
+    */
+  def createProctimeIndicatorType(): RelDataType = {
+    val originalType = createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+    canonize(
+      new TimeIndicatorRelDataType(
+        getTypeSystem,
+        originalType.asInstanceOf[BasicSqlType],
+        isEventTime = false)
+    )
+  }
+
+  /**
+    * Creates a indicator type for event-time, but with similar properties as SQL timestamp.
+    */
+  def createRowtimeIndicatorType(): RelDataType = {
+    val originalType = createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+    canonize(
+      new TimeIndicatorRelDataType(
+        getTypeSystem,
+        originalType.asInstanceOf[BasicSqlType],
+        isEventTime = true)
+    )
+  }
+
+  /**
     * Creates a struct type with the input fieldNames and input fieldTypes using FlinkTypeFactory
     *
     * @param fieldNames field names
     * @param fieldTypes field types, every element is Flink's [[TypeInformation]]
-    * @return a struct type with the input fieldNames and input fieldTypes
+    * @param rowtime optional system field to indicate event-time; the index determines the index
+    *                in the final record and might replace an existing field
+    * @param proctime optional system field to indicate processing-time; the index determines the
+    *                 index in the final record and might replace an existing field
+    * @return a struct type with the input fieldNames, input fieldTypes, and system fields
     */
-  def buildRowDataType(
-      fieldNames: Array[String],
-      fieldTypes: Array[TypeInformation[_]])
+  def buildLogicalRowType(
+      fieldNames: Seq[String],
+      fieldTypes: Seq[TypeInformation[_]],
+      rowtime: Option[(Int, String)],
+      proctime: Option[(Int, String)])
     : RelDataType = {
-    val rowDataTypeBuilder = builder
-    fieldNames
-      .zip(fieldTypes)
-      .foreach { f =>
-        rowDataTypeBuilder.add(f._1, createTypeFromTypeInfo(f._2)).nullable(true)
+    val logicalRowTypeBuilder = builder
+
+    val fields = fieldNames.zip(fieldTypes)
+
+    var totalNumberOfFields = fields.length
+    if (rowtime.isDefined) {
+      totalNumberOfFields += 1
+    }
+    if (proctime.isDefined) {
+      totalNumberOfFields += 1
+    }
+
+    var addedTimeAttributes = 0
+    for (i <- 0 until totalNumberOfFields) {
+      if (rowtime.isDefined && rowtime.get._1 == i) {
+        logicalRowTypeBuilder.add(rowtime.get._2, createRowtimeIndicatorType())
+        addedTimeAttributes += 1
+      } else if (proctime.isDefined && proctime.get._1 == i) {
+        logicalRowTypeBuilder.add(proctime.get._2, createProctimeIndicatorType())
+        addedTimeAttributes += 1
+      } else {
+        val field = fields(i - addedTimeAttributes)
+        logicalRowTypeBuilder.add(field._1, createTypeFromTypeInfo(field._2)).nullable(true)
       }
-    rowDataTypeBuilder.build
+    }
+
+    logicalRowTypeBuilder.build
   }
 
   override def createSqlType(typeName: SqlTypeName, precision: Int): RelDataType = {
@@ -178,6 +236,7 @@ object FlinkTypeFactory {
   /**
     * Converts a Calcite logical record into a Flink type information.
     */
+  @deprecated("Use the RowSchema class instead because it handles both logical and physical rows.")
   def toInternalRowTypeInfo(logicalRowType: RelDataType): TypeInformation[Row] = {
     // convert to type information
     val logicalFieldTypes = logicalRowType.getFieldList.asScala map { relDataType =>
@@ -188,6 +247,36 @@ object FlinkTypeFactory {
     new RowTypeInfo(logicalFieldTypes.toArray, logicalFieldNames.toArray)
   }
 
+  def isProctimeIndicatorType(relDataType: RelDataType): Boolean = relDataType match {
+    case ti: TimeIndicatorRelDataType if !ti.isEventTime => true
+    case _ => false
+  }
+
+  def isProctimeIndicatorType(typeInfo: TypeInformation[_]): Boolean = typeInfo match {
+    case ti: TimeIndicatorTypeInfo if !ti.isEventTime => true
+    case _ => false
+  }
+
+  def isRowtimeIndicatorType(relDataType: RelDataType): Boolean = relDataType match {
+    case ti: TimeIndicatorRelDataType if ti.isEventTime => true
+    case _ => false
+  }
+
+  def isRowtimeIndicatorType(typeInfo: TypeInformation[_]): Boolean = typeInfo match {
+    case ti: TimeIndicatorTypeInfo if ti.isEventTime => true
+    case _ => false
+  }
+
+  def isTimeIndicatorType(relDataType: RelDataType): Boolean = relDataType match {
+    case ti: TimeIndicatorRelDataType => true
+    case _ => false
+  }
+
+  def isTimeIndicatorType(typeInfo: TypeInformation[_]): Boolean = typeInfo match {
+    case ti: TimeIndicatorTypeInfo => true
+    case _ => false
+  }
+
   def toTypeInfo(relDataType: RelDataType): TypeInformation[_] = relDataType.getSqlTypeName match {
     case BOOLEAN => BOOLEAN_TYPE_INFO
     case TINYINT => BYTE_TYPE_INFO
@@ -199,6 +288,15 @@ object FlinkTypeFactory {
     case VARCHAR | CHAR => STRING_TYPE_INFO
     case DECIMAL => BIG_DEC_TYPE_INFO
 
+    // time indicators
+    case TIMESTAMP if relDataType.isInstanceOf[TimeIndicatorRelDataType] =>
+      val indicator = relDataType.asInstanceOf[TimeIndicatorRelDataType]
+      if (indicator.isEventTime) {
+        TimeIndicatorTypeInfo.ROWTIME_INDICATOR
+      } else {
+        TimeIndicatorTypeInfo.PROCTIME_INDICATOR
+      }
+
     // temporal types
     case DATE => SqlTimeTypeInfo.DATE
     case TIME => SqlTimeTypeInfo.TIME

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
new file mode 100644
index 0000000..fa2e3ee
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
@@ -0,0 +1,222 @@
+/*
+ * 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.table.calcite
+
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl, RelRecordType, StructKind}
+import org.apache.calcite.rel.logical._
+import org.apache.calcite.rel.{RelNode, RelShuttleImpl}
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.calcite.FlinkTypeFactory.isTimeIndicatorType
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType
+
+import scala.collection.JavaConversions._
+
+/**
+  * Traverses a [[RelNode]] tree and converts fields with [[TimeIndicatorRelDataType]] type. If a
+  * time attribute is accessed for a calculation, it will be materialized. Forwarding is allowed in
+  * some cases, but not all.
+  */
+class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttleImpl {
+
+  override def visit(project: LogicalProject): RelNode = {
+    // visit children and update inputs
+    val updatedProject = super.visit(project).asInstanceOf[LogicalProject]
+
+    // check if input field contains time indicator type
+    // materialize field if no time indicator is present anymore
+    // if input field is already materialized, change to timestamp type
+    val materializer = new RexTimeIndicatorMaterializer(
+      rexBuilder,
+      updatedProject.getInput.getRowType.getFieldList.map(_.getType))
+    val newProjects = updatedProject.getProjects.map(_.accept(materializer))
+
+    // copy project
+    updatedProject.copy(
+      updatedProject.getTraitSet,
+      updatedProject.getInput,
+      newProjects,
+      buildRowType(updatedProject.getRowType.getFieldNames, newProjects.map(_.getType))
+    )
+  }
+
+  override def visit(filter: LogicalFilter): RelNode = {
+    // visit children and update inputs
+    val updatedFilter = super.visit(filter).asInstanceOf[LogicalFilter]
+
+    // check if input field contains time indicator type
+    // materialize field if no time indicator is present anymore
+    // if input field is already materialized, change to timestamp type
+    val materializer = new RexTimeIndicatorMaterializer(
+      rexBuilder,
+      updatedFilter.getInput.getRowType.getFieldList.map(_.getType))
+    val newCondition = updatedFilter.getCondition.accept(materializer)
+
+    // copy filter
+    updatedFilter.copy(
+      updatedFilter.getTraitSet,
+      updatedFilter.getInput,
+      newCondition
+    )
+  }
+
+  override def visit(union: LogicalUnion): RelNode = {
+    // visit children and update inputs
+    val updatedUnion = super.visit(union).asInstanceOf[LogicalUnion]
+
+    // make sure that time indicator types match
+    val inputTypes = updatedUnion.getInputs.map(_.getRowType)
+
+    val head = inputTypes.head.getFieldList.map(_.getType)
+
+    val isValid = inputTypes.forall { t =>
+      val fieldTypes = t.getFieldList.map(_.getType)
+
+      fieldTypes.zip(head).forall { case (l, r) =>
+        // check if time indicators match
+        if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) {
+          val leftTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime
+          val rightTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime
+          leftTime == rightTime
+        }
+        // one side is not an indicator
+        else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) {
+          false
+        }
+        // uninteresting types
+        else {
+          true
+        }
+      }
+    }
+
+    if (!isValid) {
+      throw new ValidationException(
+        "Union fields with time attributes have different types.")
+    }
+
+    updatedUnion
+  }
+
+  override def visit(other: RelNode): RelNode = other match {
+    case scan: LogicalTableFunctionScan if
+        stack.size() > 0 && stack.peek().isInstanceOf[LogicalCorrelate] =>
+      // visit children and update inputs
+      val updatedScan = super.visit(scan).asInstanceOf[LogicalTableFunctionScan]
+
+      val correlate = stack.peek().asInstanceOf[LogicalCorrelate]
+
+      // check if input field contains time indicator type
+      // materialize field if no time indicator is present anymore
+      // if input field is already materialized, change to timestamp type
+      val materializer = new RexTimeIndicatorMaterializer(
+        rexBuilder,
+        correlate.getInputs.get(0).getRowType.getFieldList.map(_.getType))
+      val newCall = updatedScan.getCall.accept(materializer)
+
+      // copy scan
+      updatedScan.copy(
+        updatedScan.getTraitSet,
+        updatedScan.getInputs,
+        newCall,
+        updatedScan.getElementType,
+        updatedScan.getRowType,
+        updatedScan.getColumnMappings
+      )
+
+    case _ =>
+      super.visit(other)
+  }
+
+  private def buildRowType(names: Seq[String], types: Seq[RelDataType]): RelDataType = {
+    val fields = names.zipWithIndex.map { case (name, idx) =>
+      new RelDataTypeFieldImpl(name, idx, types(idx))
+    }
+    new RelRecordType(StructKind.FULLY_QUALIFIED, fields)
+  }
+}
+
+class RexTimeIndicatorMaterializer(
+    private val rexBuilder: RexBuilder,
+    private val input: Seq[RelDataType])
+  extends RexShuttle {
+
+  val timestamp = rexBuilder
+    .getTypeFactory
+    .asInstanceOf[FlinkTypeFactory]
+    .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+
+  override def visitInputRef(inputRef: RexInputRef): RexNode = {
+    // reference is interesting
+    if (isTimeIndicatorType(inputRef.getType)) {
+      val resolvedRefType = input(inputRef.getIndex)
+      // input is a valid time indicator
+      if (isTimeIndicatorType(resolvedRefType)) {
+        inputRef
+      }
+      // input has been materialized
+      else {
+        new RexInputRef(inputRef.getIndex, resolvedRefType)
+      }
+    }
+    // reference is a regular field
+    else {
+      super.visitInputRef(inputRef)
+    }
+  }
+
+  override def visitCall(call: RexCall): RexNode = {
+    val updatedCall = super.visitCall(call).asInstanceOf[RexCall]
+
+    // skip materialization for special operators
+    updatedCall.getOperator match {
+      case SqlStdOperatorTable.SESSION | SqlStdOperatorTable.HOP | SqlStdOperatorTable.TUMBLE =>
+        return updatedCall
+
+      case _ => // do nothing
+    }
+
+    // materialize operands with time indicators
+    val materializedOperands = updatedCall.getOperands.map { o =>
+      if (isTimeIndicatorType(o.getType)) {
+        rexBuilder.makeCall(TimeMaterializationSqlFunction, o)
+      } else {
+        o
+      }
+    }
+
+    // remove time indicator return type
+    if (isTimeIndicatorType(updatedCall.getType)) {
+      updatedCall.clone(timestamp, materializedOperands)
+    } else {
+      updatedCall.clone(updatedCall.getType, materializedOperands)
+    }
+  }
+}
+
+object RelTimeIndicatorConverter {
+
+  def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = {
+    val converter = new RelTimeIndicatorConverter(rexBuilder)
+    rootRel.accept(converter)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
index 5bb3b0e..25addbc 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
@@ -23,7 +23,6 @@ import java.lang.{Iterable => JIterable}
 import java.math.{BigDecimal => JBigDecimal}
 
 import org.apache.calcite.avatica.util.DateTimeUtils
-import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rex._
 import org.apache.calcite.sql.SqlOperator
 import org.apache.calcite.sql.`type`.SqlTypeName._
@@ -42,8 +41,8 @@ import org.apache.flink.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.codegen.calls.FunctionGenerator
 import org.apache.flink.table.codegen.calls.ScalarOperators._
-import org.apache.flink.table.functions.{AggregateFunction, FunctionContext, UserDefinedFunction}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
+import org.apache.flink.table.functions.{AggregateFunction, FunctionContext, TimeMaterializationSqlFunction, UserDefinedFunction}
 import org.apache.flink.table.runtime.TableFunctionCollector
 import org.apache.flink.table.typeutils.TypeCheckUtils._
 import org.apache.flink.types.Row
@@ -59,19 +58,18 @@ import scala.collection.mutable
   * @param nullableInput input(s) can be null.
   * @param input1 type information about the first input of the Function
   * @param input2 type information about the second input if the Function is binary
-  * @param input1PojoFieldMapping additional mapping information if input1 is a POJO (POJO types
-  *                              have no deterministic field order).
-  * @param input2PojoFieldMapping additional mapping information if input2 is a POJO (POJO types
-  *                              have no deterministic field order).
-  *
+  * @param input1FieldMapping additional mapping information for input1
+  *   (e.g. POJO types have no deterministic field order and some input fields might not be read)
+  * @param input2FieldMapping additional mapping information for input2
+  *   (e.g. POJO types have no deterministic field order and some input fields might not be read)
   */
 class CodeGenerator(
-   config: TableConfig,
-   nullableInput: Boolean,
-   input1: TypeInformation[_ <: Any],
-   input2: Option[TypeInformation[_ <: Any]] = None,
-   input1PojoFieldMapping: Option[Array[Int]] = None,
-   input2PojoFieldMapping: Option[Array[Int]] = None)
+    config: TableConfig,
+    nullableInput: Boolean,
+    input1: TypeInformation[_ <: Any],
+    input2: Option[TypeInformation[_ <: Any]] = None,
+    input1FieldMapping: Option[Array[Int]] = None,
+    input2FieldMapping: Option[Array[Int]] = None)
   extends RexVisitor[GeneratedExpression] {
 
   // check if nullCheck is enabled when inputs can be null
@@ -82,7 +80,7 @@ class CodeGenerator(
   // check for POJO input1 mapping
   input1 match {
     case pt: PojoTypeInfo[_] =>
-      input1PojoFieldMapping.getOrElse(
+      input1FieldMapping.getOrElse(
         throw new CodeGenException("No input mapping is specified for input1 of type POJO."))
     case _ => // ok
   }
@@ -90,11 +88,24 @@ class CodeGenerator(
   // check for POJO input2 mapping
   input2 match {
     case Some(pt: PojoTypeInfo[_]) =>
-      input2PojoFieldMapping.getOrElse(
+      input2FieldMapping.getOrElse(
         throw new CodeGenException("No input mapping is specified for input2 of type POJO."))
     case _ => // ok
   }
 
+  private val input1Mapping = input1FieldMapping match {
+    case Some(mapping) => mapping
+    case _ => (0 until input1.getArity).toArray
+  }
+
+  private val input2Mapping = input2FieldMapping match {
+    case Some(mapping) => mapping
+    case _ => input2 match {
+      case Some(input) => (0 until input.getArity).toArray
+      case _ => Array[Int]()
+    }
+  }
+
   /**
     * A code generator for generating unary Flink
     * [[org.apache.flink.api.common.functions.Function]]s with one input.
@@ -102,15 +113,15 @@ class CodeGenerator(
     * @param config configuration that determines runtime behavior
     * @param nullableInput input(s) can be null.
     * @param input type information about the input of the Function
-    * @param inputPojoFieldMapping additional mapping information necessary if input is a
-    *                              POJO (POJO types have no deterministic field order).
+    * @param inputFieldMapping additional mapping information necessary for input
+    *   (e.g. POJO types have no deterministic field order and some input fields might not be read)
     */
   def this(
       config: TableConfig,
       nullableInput: Boolean,
       input: TypeInformation[Any],
-      inputPojoFieldMapping: Array[Int]) =
-    this(config, nullableInput, input, None, Some(inputPojoFieldMapping))
+      inputFieldMapping: Array[Int]) =
+    this(config, nullableInput, input, None, Some(inputFieldMapping))
 
   /**
     * A code generator for generating Flink input formats.
@@ -249,7 +260,7 @@ class CodeGenerator(
     * @param name        Class name of the function.
     *                    Does not need to be unique but has to be a valid Java class identifier.
     * @param generator   The code generator instance
-    * @param inputType   Input row type
+    * @param physicalInputTypes Physical input row types
     * @param aggregates  All aggregate functions
     * @param aggFields   Indexes of the input fields for all aggregate functions
     * @param aggMapping  The mapping of aggregates to output fields
@@ -270,7 +281,7 @@ class CodeGenerator(
   def generateAggregations(
       name: String,
       generator: CodeGenerator,
-      inputType: RelDataType,
+      physicalInputTypes: Seq[TypeInformation[_]],
       aggregates: Array[AggregateFunction[_ <: Any, _ <: Any]],
       aggFields: Array[Array[Int]],
       aggMapping: Array[Int],
@@ -295,8 +306,7 @@ class CodeGenerator(
     val accTypes = accTypeClasses.map(_.getCanonicalName)
 
     // get java classes of input fields
-    val javaClasses = inputType.getFieldList
-      .map(f => FlinkTypeFactory.toTypeInfo(f.getType).getTypeClass)
+    val javaClasses = physicalInputTypes.map(t => t.getTypeClass)
     // get parameter lists for aggregation functions
     val parameters = aggFields.map { inFields =>
       val fields = for (f <- inFields) yield
@@ -844,12 +854,12 @@ class CodeGenerator(
       returnType: TypeInformation[_ <: Any],
       resultFieldNames: Seq[String])
     : GeneratedExpression = {
-    val input1AccessExprs = for (i <- 0 until input1.getArity)
-      yield generateInputAccess(input1, input1Term, i, input1PojoFieldMapping)
+    val input1AccessExprs = for (i <- 0 until input1.getArity if input1Mapping.contains(i))
+      yield generateInputAccess(input1, input1Term, i, input1Mapping)
 
     val input2AccessExprs = input2 match {
-      case Some(ti) => for (i <- 0 until ti.getArity)
-        yield generateInputAccess(ti, input2Term, i, input2PojoFieldMapping)
+      case Some(ti) => for (i <- 0 until ti.getArity if input2Mapping.contains(i))
+        yield generateInputAccess(ti, input2Term, i, input2Mapping)
       case None => Seq() // add nothing
     }
 
@@ -861,14 +871,14 @@ class CodeGenerator(
     */
   def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = {
     val input1AccessExprs = for (i <- 0 until input1.getArity)
-      yield generateInputAccess(input1, input1Term, i, input1PojoFieldMapping)
+      yield generateInputAccess(input1, input1Term, i, input1Mapping)
 
     val input2AccessExprs = input2 match {
-      case Some(ti) => for (i <- 0 until ti.getArity)
+      case Some(ti) => for (i <- 0 until ti.getArity if input2Mapping.contains(i))
         // use generateFieldAccess instead of generateInputAccess to avoid the generated table
         // function's field access code is put on the top of function body rather than
         // the while loop
-        yield generateFieldAccess(ti, input2Term, i, input2PojoFieldMapping)
+        yield generateFieldAccess(ti, input2Term, i, input2Mapping)
       case None => throw new CodeGenException("Type information of input2 must not be null.")
     }
     (input1AccessExprs, input2AccessExprs)
@@ -1123,11 +1133,11 @@ class CodeGenerator(
   override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = {
     // if inputRef index is within size of input1 we work with input1, input2 otherwise
     val input = if (inputRef.getIndex < input1.getArity) {
-      (input1, input1Term, input1PojoFieldMapping)
+      (input1, input1Term, input1Mapping)
     } else {
       (input2.getOrElse(throw new CodeGenException("Invalid input access.")),
         input2Term,
-        input2PojoFieldMapping)
+        input2Mapping)
     }
 
     val index = if (input._2 == input1Term) {
@@ -1146,7 +1156,7 @@ class CodeGenerator(
       refExpr.resultType,
       refExpr.resultTerm,
       index,
-      input1PojoFieldMapping)
+      input1Mapping)
 
     val resultTerm = newName("result")
     val nullTerm = newName("isNull")
@@ -1302,6 +1312,11 @@ class CodeGenerator(
     throw new CodeGenException("Dynamic parameter references are not supported yet.")
 
   override def visitCall(call: RexCall): GeneratedExpression = {
+    // time materialization is not implemented yet
+    if (call.getOperator == TimeMaterializationSqlFunction) {
+      throw new CodeGenException("Access to time attributes is not possible yet.")
+    }
+
     val operands = call.getOperands.map(_.accept(this))
     val resultType = FlinkTypeFactory.toTypeInfo(call.getType)
 
@@ -1546,7 +1561,7 @@ class CodeGenerator(
       inputType: TypeInformation[_ <: Any],
       inputTerm: String,
       index: Int,
-      pojoFieldMapping: Option[Array[Int]])
+      fieldMapping: Array[Int])
     : GeneratedExpression = {
     // if input has been used before, we can reuse the code that
     // has already been generated
@@ -1558,9 +1573,9 @@ class CodeGenerator(
       // generate input access and unboxing if necessary
       case None =>
         val expr = if (nullableInput) {
-          generateNullableInputFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
+          generateNullableInputFieldAccess(inputType, inputTerm, index, fieldMapping)
         } else {
-          generateFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
+          generateFieldAccess(inputType, inputTerm, index, fieldMapping)
         }
 
         reusableInputUnboxingExprs((inputTerm, index)) = expr
@@ -1574,7 +1589,7 @@ class CodeGenerator(
       inputType: TypeInformation[_ <: Any],
       inputTerm: String,
       index: Int,
-      pojoFieldMapping: Option[Array[Int]])
+      fieldMapping: Array[Int])
     : GeneratedExpression = {
     val resultTerm = newName("result")
     val nullTerm = newName("isNull")
@@ -1582,7 +1597,7 @@ class CodeGenerator(
     val fieldType = inputType match {
       case ct: CompositeType[_] =>
         val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) {
-          pojoFieldMapping.get(index)
+          fieldMapping(index)
         }
         else {
           index
@@ -1593,7 +1608,7 @@ class CodeGenerator(
     }
     val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType)
     val defaultValue = primitiveDefaultValue(fieldType)
-    val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
+    val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index, fieldMapping)
 
     val inputCheckCode =
       s"""
@@ -1617,12 +1632,12 @@ class CodeGenerator(
       inputType: TypeInformation[_],
       inputTerm: String,
       index: Int,
-      pojoFieldMapping: Option[Array[Int]])
+      fieldMapping: Array[Int])
     : GeneratedExpression = {
     inputType match {
       case ct: CompositeType[_] =>
-        val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]] && pojoFieldMapping.nonEmpty) {
-          pojoFieldMapping.get(index)
+        val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) {
+          fieldMapping(index)
         }
         else {
           index

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
index d9f394b..64280c8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
@@ -29,7 +29,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
-import org.apache.flink.table.functions.{EventTimeExtractor, ProcTimeExtractor}
 import org.apache.flink.table.functions.utils.{ScalarSqlFunction, TableSqlFunction}
 
 import scala.collection.mutable
@@ -496,15 +495,6 @@ object FunctionGenerator {
         )
       )
 
-    // generate a constant for time indicator functions.
-    // this is a temporary solution and will be removed when FLINK-5884 is implemented.
-    case ProcTimeExtractor | EventTimeExtractor =>
-      Some(new CallGenerator {
-        override def generate(codeGenerator: CodeGenerator, operands: Seq[GeneratedExpression]) = {
-          GeneratedExpression("0L", "false", "", SqlTimeTypeInfo.TIMESTAMP)
-        }
-      })
-
     // built-in scalar function
     case _ =>
       sqlFunctions.get((sqlOperator, operandTypes))

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
index 4b5781f..08abc8f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
@@ -28,10 +28,49 @@ import org.apache.calcite.rex.{RexBuilder, RexNode}
 import org.apache.calcite.sql.fun.SqlStdOperatorTable
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
 import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
+import org.apache.flink.streaming.api.windowing.time.{Time => FlinkTime}
 
 object ExpressionUtils {
 
+  private[flink] def isTimeIntervalLiteral(expr: Expression): Boolean = expr match {
+    case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) => true
+    case _ => false
+  }
+
+  private[flink] def isRowCountLiteral(expr: Expression): Boolean = expr match {
+    case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) => true
+    case _ => false
+  }
+
+  private[flink] def isTimeAttribute(expr: Expression): Boolean = expr match {
+    case r: ResolvedFieldReference if FlinkTypeFactory.isTimeIndicatorType(r.resultType) => true
+    case _ => false
+  }
+
+  private[flink] def isRowtimeAttribute(expr: Expression): Boolean = expr match {
+    case r: ResolvedFieldReference if FlinkTypeFactory.isRowtimeIndicatorType(r.resultType) => true
+    case _ => false
+  }
+
+  private[flink] def isProctimeAttribute(expr: Expression): Boolean = expr match {
+    case r: ResolvedFieldReference if FlinkTypeFactory.isProctimeIndicatorType(r.resultType) =>
+      true
+    case _ => false
+  }
+
+  private[flink] def toTime(expr: Expression): FlinkTime = expr match {
+    case Literal(value: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+      FlinkTime.milliseconds(value)
+    case _ => throw new IllegalArgumentException()
+  }
+
+  private[flink] def toLong(expr: Expression): Long = expr match {
+    case Literal(value: Long, RowIntervalTypeInfo.INTERVAL_ROWS) => value
+    case _ => throw new IllegalArgumentException()
+  }
+
   private[flink] def toMonthInterval(expr: Expression, multiplier: Int): Expression = expr match {
     case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
       Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MONTHS)

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
index 5f7204a..13f8a11 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
@@ -110,18 +110,11 @@ case class OverCall(
     val aggExprs = agg.asInstanceOf[Aggregation].children.map(_.toRexNode(relBuilder)).asJava
 
     // assemble order by key
-    val orderKey = orderBy match {
-      case _: RowTime =>
-        new RexFieldCollation(relBuilder.call(EventTimeExtractor), Set[SqlKind]().asJava)
-      case _: ProcTime =>
-        new RexFieldCollation(relBuilder.call(ProcTimeExtractor), Set[SqlKind]().asJava)
-      case _ =>
-        throw new ValidationException("Invalid OrderBy expression.")
-    }
+    val orderKey = new RexFieldCollation(orderBy.toRexNode, Set[SqlKind]().asJava)
     val orderKeys = ImmutableList.of(orderKey)
 
     // assemble partition by keys
-    val partitionKeys = partitionBy.map(_.toRexNode(relBuilder)).asJava
+    val partitionKeys = partitionBy.map(_.toRexNode).asJava
 
     // assemble bounds
     val isPhysical: Boolean = preceding.resultType.isInstanceOf[RowIntervalTypeInfo]
@@ -249,6 +242,11 @@ case class OverCall(
         return ValidationFailure("Preceding and following must be of same interval type.")
     }
 
+    // check time field
+    if (!ExpressionUtils.isTimeAttribute(orderBy)) {
+      return ValidationFailure("Ordering must be defined on a time attribute.")
+    }
+
     ValidationSuccess
   }
 }


[12/15] flink git commit: [FLINK-6091] [table] Implement and turn on retraction for non-windowed aggregates.

Posted by fh...@apache.org.
[FLINK-6091] [table] Implement and turn on retraction for non-windowed aggregates.


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

Branch: refs/heads/master
Commit: 27bf4cab7141ccbc7e8effe03559c50bbb3f9707
Parents: dc54abc
Author: Hequn Cheng <ch...@gmail.com>
Authored: Tue Apr 18 16:54:09 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:55 2017 +0200

----------------------------------------------------------------------
 .../flink/table/api/BatchTableEnvironment.scala |  63 +++++-
 .../table/api/StreamTableEnvironment.scala      |  64 +++++-
 .../flink/table/api/TableEnvironment.scala      |  65 +++---
 .../flink/table/plan/nodes/CommonCalc.scala     |  32 ++-
 .../table/plan/nodes/CommonCorrelate.scala      | 139 +++++--------
 .../flink/table/plan/nodes/CommonScan.scala     |   7 +-
 .../table/plan/nodes/dataset/BatchScan.scala    |   2 +-
 .../table/plan/nodes/dataset/DataSetCalc.scala  |  16 +-
 .../plan/nodes/dataset/DataSetCorrelate.scala   |  29 ++-
 .../plan/nodes/dataset/DataSetValues.scala      |   2 +-
 .../plan/nodes/datastream/DataStreamCalc.scala  |  26 +--
 .../nodes/datastream/DataStreamCorrelate.scala  |  40 +++-
 .../datastream/DataStreamGroupAggregate.scala   |  24 ++-
 .../DataStreamGroupWindowAggregate.scala        |  52 +++--
 .../datastream/DataStreamOverAggregate.scala    |  72 ++++---
 .../plan/nodes/datastream/DataStreamRel.scala   |   6 +-
 .../plan/nodes/datastream/DataStreamScan.scala  |   7 +-
 .../plan/nodes/datastream/DataStreamUnion.scala |   4 +-
 .../nodes/datastream/DataStreamValues.scala     |  13 +-
 .../plan/nodes/datastream/StreamScan.scala      |  27 ++-
 .../datastream/StreamTableSourceScan.scala      |   8 +-
 .../plan/nodes/logical/FlinkLogicalCalc.scala   |   2 +-
 .../datastream/DataStreamRetractionRules.scala  |  16 +-
 .../runtime/CRowCorrelateFlatMapRunner.scala    |  83 ++++++++
 .../flink/table/runtime/CRowFlatMapRunner.scala |  72 +++++++
 .../table/runtime/CRowInputMapRunner.scala      |  57 ++++++
 .../table/runtime/CRowOutputMapRunner.scala     |  60 ++++++
 .../table/runtime/CRowWrappingCollector.scala   |  41 ++++
 .../flink/table/runtime/FlatMapRunner.scala     |  17 +-
 .../aggregate/AggregateAggFunction.scala        |  15 +-
 .../table/runtime/aggregate/AggregateUtil.scala |  48 +++--
 ...SetSessionWindowAggReduceGroupFunction.scala |   4 +-
 ...taSetSlideWindowAggReduceGroupFunction.scala |   4 +-
 ...TumbleTimeWindowAggReduceGroupFunction.scala |   4 +-
 .../aggregate/GroupAggProcessFunction.scala     |  58 ++++--
 ...rementalAggregateAllTimeWindowFunction.scala |   7 +-
 .../IncrementalAggregateAllWindowFunction.scala |  11 +-
 ...IncrementalAggregateTimeWindowFunction.scala |   7 +-
 .../IncrementalAggregateWindowFunction.scala    |  13 +-
 .../aggregate/ProcTimeBoundedRangeOver.scala    |  30 +--
 .../aggregate/ProcTimeBoundedRowsOver.scala     |  26 ++-
 .../ProcTimeUnboundedNonPartitionedOver.scala   |  23 ++-
 .../ProcTimeUnboundedPartitionedOver.scala      |  22 +-
 .../aggregate/RowTimeBoundedRangeOver.scala     |  36 ++--
 .../aggregate/RowTimeBoundedRowsOver.scala      |  30 +--
 .../aggregate/RowTimeUnboundedOver.scala        |  50 ++---
 .../aggregate/TimeWindowPropertyCollector.scala |  34 ++-
 .../runtime/io/CRowValuesInputFormat.scala      |  59 ++++++
 .../table/runtime/io/ValuesInputFormat.scala    |  17 +-
 .../apache/flink/table/runtime/types/CRow.scala |  55 +++++
 .../table/runtime/types/CRowComparator.scala    |  83 ++++++++
 .../table/runtime/types/CRowSerializer.scala    |  78 +++++++
 .../table/runtime/types/CRowTypeInfo.scala      |  98 +++++++++
 .../apache/flink/table/sinks/CsvTableSink.scala |   2 +
 .../flink/table/TableEnvironmentTest.scala      |  72 ++++++-
 .../scala/batch/TableEnvironmentITCase.scala    |  20 ++
 .../api/scala/stream/RetractionITCase.scala     | 205 +++++++++++++++++++
 .../api/scala/stream/TableSinkITCase.scala      |   2 +-
 .../api/scala/stream/utils/StreamITCase.scala   |  11 +-
 ...ProcessingOverRangeProcessFunctionTest.scala | 105 +++++-----
 .../runtime/types/CRowComparatorTest.scala      |  61 ++++++
 .../table/utils/MockTableEnvironment.scala      |   8 +
 62 files changed, 1813 insertions(+), 531 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index 02c6063..f7955f0 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -26,16 +26,20 @@ import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql2rel.RelDecorrelator
 import org.apache.calcite.tools.RuleSet
+import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.io.DiscardingOutputFormat
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
+import org.apache.flink.configuration.Configuration
 import org.apache.flink.table.explain.PlanJsonParser
 import org.apache.flink.table.expressions.{Expression, RowtimeAttribute, TimeAttribute}
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.dataset.DataSetRel
 import org.apache.flink.table.plan.rules.FlinkRuleSets
 import org.apache.flink.table.plan.schema.{DataSetTable, TableSourceTable}
+import org.apache.flink.table.runtime.MapRunner
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.table.sinks.{BatchTableSink, TableSink}
 import org.apache.flink.table.sources.{BatchTableSource, TableSource}
 import org.apache.flink.types.Row
@@ -128,6 +132,56 @@ abstract class BatchTableEnvironment(
   }
 
   /**
+    * Creates a final converter that maps the internal row type to external type.
+    *
+    * @param physicalTypeInfo the input of the sink
+    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
+    * @param requestedTypeInfo the output type of the sink
+    * @param functionName name of the map function. Must not be unique but has to be a
+    *                     valid Java class identifier.
+    */
+  override protected def getConversionMapper[IN, OUT](
+      physicalTypeInfo: TypeInformation[IN],
+      logicalRowType: RelDataType,
+      requestedTypeInfo: TypeInformation[OUT],
+      functionName: String):
+    Option[MapFunction[IN, OUT]] = {
+
+    if (requestedTypeInfo.getTypeClass == classOf[Row]) {
+      // Row to Row, no conversion needed
+      None
+    } else if (requestedTypeInfo.getTypeClass == classOf[CRow]) {
+      // Row to CRow, only needs to be wrapped
+      Some(
+        new RichMapFunction[Row, CRow] {
+          private var outCRow: CRow = _
+          override def open(parameters: Configuration): Unit = outCRow = new CRow(null, true)
+          override def map(value: Row): CRow = {
+            outCRow.row = value
+            outCRow
+          }
+        }.asInstanceOf[MapFunction[IN, OUT]]
+      )
+    } else {
+      // some type that is neither Row or CRow
+
+      val converterFunction = generateRowConverterFunction[OUT](
+        physicalTypeInfo.asInstanceOf[TypeInformation[Row]],
+        logicalRowType,
+        requestedTypeInfo,
+        functionName
+      )
+
+      val mapFunction = new MapRunner[IN, OUT](
+        converterFunction.name,
+        converterFunction.code,
+        converterFunction.returnType)
+
+      Some(mapFunction)
+    }
+  }
+
+  /**
     * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
     * the result of the given [[Table]].
     *
@@ -293,10 +347,15 @@ abstract class BatchTableEnvironment(
     logicalPlan match {
       case node: DataSetRel =>
         val plan = node.translateToPlan(this)
-        val conversion = sinkConversion(plan.getType, logicalType, tpe, "DataSetSinkConversion")
+        val conversion =
+          getConversionMapper(plan.getType, logicalType, tpe, "DataSetSinkConversion")
         conversion match {
           case None => plan.asInstanceOf[DataSet[A]] // no conversion necessary
-          case Some(mapFunction) => plan.map(mapFunction).name(s"to: $tpe")
+          case Some(mapFunction: MapFunction[Row, A]) =>
+            plan.map(mapFunction)
+              .returns(tpe)
+              .name(s"to: ${tpe.getTypeClass.getSimpleName}")
+              .asInstanceOf[DataSet[A]]
         }
 
       case _ =>

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index dd2c09d..0632a47 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -28,6 +28,7 @@ import org.apache.calcite.sql2rel.RelDecorrelator
 import org.apache.calcite.tools.{RuleSet, RuleSets}
 import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
 import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.common.functions.MapFunction
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
@@ -37,7 +38,9 @@ import org.apache.flink.table.expressions.{Expression, ProctimeAttribute, Rowtim
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
 import org.apache.flink.table.plan.rules.FlinkRuleSets
-import org.apache.flink.table.plan.schema.{DataStreamTable, StreamTableSourceTable, TableSourceTable}
+import org.apache.flink.table.plan.schema.{DataStreamTable, StreamTableSourceTable}
+import org.apache.flink.table.runtime.CRowInputMapRunner
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.apache.flink.table.sinks.{StreamTableSink, TableSink}
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
 import org.apache.flink.table.typeutils.TypeCheckUtils
@@ -132,6 +135,52 @@ abstract class StreamTableEnvironment(
     }
   }
 
+
+  /**
+    * Creates a final converter that maps the internal row type to external type.
+    *
+    * @param physicalTypeInfo the input of the sink
+    * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
+    * @param requestedTypeInfo the output type of the sink
+    * @param functionName name of the map function. Must not be unique but has to be a
+    *                     valid Java class identifier.
+    */
+  override protected def getConversionMapper[IN, OUT](
+      physicalTypeInfo: TypeInformation[IN],
+      logicalRowType: RelDataType,
+      requestedTypeInfo: TypeInformation[OUT],
+      functionName: String):
+  Option[MapFunction[IN, OUT]] = {
+
+    if (requestedTypeInfo.getTypeClass == classOf[CRow]) {
+      // CRow to CRow, no conversion needed
+      None
+    } else if (requestedTypeInfo.getTypeClass == classOf[Row]) {
+      // CRow to Row, only needs to be unwrapped
+      Some(
+        new MapFunction[CRow, Row] {
+          override def map(value: CRow): Row = value.row
+        }.asInstanceOf[MapFunction[IN, OUT]]
+      )
+    } else {
+      // Some type that is neither CRow nor Row
+
+      val converterFunction = generateRowConverterFunction[OUT](
+        physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType,
+        logicalRowType,
+        requestedTypeInfo,
+        functionName
+      )
+
+      Some(new CRowInputMapRunner[OUT](
+        converterFunction.name,
+        converterFunction.code,
+        converterFunction.returnType)
+        .asInstanceOf[MapFunction[IN, OUT]])
+
+    }
+  }
+
   /**
     * Registers a [[DataStream]] as a table under a given name in the [[TableEnvironment]]'s
     * catalog.
@@ -377,10 +426,15 @@ abstract class StreamTableEnvironment(
     logicalPlan match {
       case node: DataStreamRel =>
         val plan = node.translateToPlan(this)
-        val conversion = sinkConversion(plan.getType, logicalType, tpe, "DataStreamSinkConversion")
+        val conversion =
+          getConversionMapper(plan.getType, logicalType, tpe, "DataStreamSinkConversion")
         conversion match {
           case None => plan.asInstanceOf[DataStream[A]] // no conversion necessary
-          case Some(mapFunction) => plan.map(mapFunction).name(s"to: $tpe")
+          case Some(mapFunction: MapFunction[CRow, A]) =>
+            plan.map(mapFunction)
+              .returns(tpe)
+              .name(s"to: ${tpe.getTypeClass.getSimpleName}")
+              .asInstanceOf[DataStream[A]]
         }
 
       case _ =>
@@ -398,9 +452,9 @@ abstract class StreamTableEnvironment(
   def explain(table: Table): String = {
     val ast = table.getRelNode
     val optimizedPlan = optimize(ast)
-    val dataStream = translate[Row](
+    val dataStream = translate[CRow](
       optimizedPlan,
-      ast.getRowType)(new GenericTypeInfo(classOf[Row]))
+      ast.getRowType)(new GenericTypeInfo(classOf[CRow]))
 
     val env = dataStream.getExecutionEnvironment
     val jsonSqlPlan = env.getExecutionPlan

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 9ed5000..d27db1e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -48,7 +48,7 @@ import org.apache.flink.table.api.java.{BatchTableEnvironment => JavaBatchTableE
 import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv}
 import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem}
 import org.apache.flink.table.catalog.{ExternalCatalog, ExternalCatalogSchema}
-import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer, GeneratedFunction}
 import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{ScalarFunction, TableFunction, AggregateFunction}
@@ -59,7 +59,7 @@ import org.apache.flink.table.plan.cost.DataSetCostFactory
 import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode}
 import org.apache.flink.table.plan.rules.FlinkRuleSets
 import org.apache.flink.table.plan.schema.RelTable
-import org.apache.flink.table.runtime.MapRunner
+import org.apache.flink.table.runtime.types.CRowTypeInfo
 import org.apache.flink.table.sinks.TableSink
 import org.apache.flink.table.sources.{DefinedFieldNames, TableSource}
 import org.apache.flink.table.validate.FunctionCatalog
@@ -644,6 +644,18 @@ abstract class TableEnvironment(val config: TableConfig) {
           case _ => throw new TableException(
             "Field reference expression or alias on field expression expected.")
         }
+      case cr: CRowTypeInfo =>
+        exprs.zipWithIndex.map {
+          case (UnresolvedFieldReference(name), idx) => (idx, name)
+          case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
+            val idx = cr.getFieldIndex(origName)
+            if (idx < 0) {
+              throw new TableException(s"$origName is not a field of type $cr")
+            }
+            (idx, name)
+          case _ => throw new TableException(
+            "Field reference expression or alias on field expression expected.")
+        }
       case c: CaseClassTypeInfo[A] =>
         exprs.zipWithIndex flatMap {
           case (UnresolvedFieldReference(name), idx) =>
@@ -694,37 +706,38 @@ abstract class TableEnvironment(val config: TableConfig) {
   /**
     * Creates a final converter that maps the internal row type to external type.
     *
-    * @param physicalRowTypeInfo the input of the sink
+    * @param physicalTypeInfo the input of the sink
     * @param logicalRowType the logical type with correct field names (esp. for POJO field mapping)
     * @param requestedTypeInfo the output type of the sink
     * @param functionName name of the map function. Must not be unique but has to be a
     *                     valid Java class identifier.
     */
-  protected def sinkConversion[T](
-      physicalRowTypeInfo: TypeInformation[Row],
+  protected def getConversionMapper[IN, OUT](
+      physicalTypeInfo: TypeInformation[IN],
+      logicalRowType: RelDataType,
+      requestedTypeInfo: TypeInformation[OUT],
+      functionName: String):
+    Option[MapFunction[IN, OUT]]
+
+  protected def generateRowConverterFunction[OUT](
+      inputTypeInfo: TypeInformation[Row],
       logicalRowType: RelDataType,
-      requestedTypeInfo: TypeInformation[T],
-      functionName: String)
-    : Option[MapFunction[Row, T]] = {
+      requestedTypeInfo: TypeInformation[OUT],
+      functionName: String):
+    GeneratedFunction[MapFunction[Row, OUT], OUT] = {
 
     // validate that at least the field types of physical and logical type match
     // we do that here to make sure that plan translation was correct
     val logicalRowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(logicalRowType)
-    if (physicalRowTypeInfo != logicalRowTypeInfo) {
+    if (logicalRowTypeInfo != inputTypeInfo) {
       throw TableException("The field types of physical and logical row types do not match." +
         "This is a bug and should not happen. Please file an issue.")
     }
 
-    // requested type is a generic Row, no conversion needed
-    if (requestedTypeInfo.isInstanceOf[GenericTypeInfo[_]] &&
-          requestedTypeInfo.getTypeClass == classOf[Row]) {
-      return None
-    }
-
     // convert to type information
-    val logicalFieldTypes = logicalRowType.getFieldList.asScala map { relDataType =>
-      FlinkTypeFactory.toTypeInfo(relDataType.getType)
-    }
+    val logicalFieldTypes = logicalRowType.getFieldList.asScala
+      .map(t => FlinkTypeFactory.toTypeInfo(t.getType))
+
     // field names
     val logicalFieldNames = logicalRowType.getFieldNames.asScala
 
@@ -732,8 +745,8 @@ abstract class TableEnvironment(val config: TableConfig) {
     if (requestedTypeInfo.getArity != logicalFieldTypes.length) {
       throw new TableException("Arity of result does not match requested type.")
     }
-    requestedTypeInfo match {
 
+    requestedTypeInfo match {
       // POJO type requested
       case pt: PojoTypeInfo[_] =>
         logicalFieldNames.zip(logicalFieldTypes) foreach {
@@ -780,7 +793,7 @@ abstract class TableEnvironment(val config: TableConfig) {
     val generator = new CodeGenerator(
       config,
       false,
-      physicalRowTypeInfo,
+      inputTypeInfo,
       None,
       None)
 
@@ -794,20 +807,12 @@ abstract class TableEnvironment(val config: TableConfig) {
          |return ${conversion.resultTerm};
          |""".stripMargin
 
-    val genFunction = generator.generateFunction(
+    generator.generateFunction(
       functionName,
-      classOf[MapFunction[Row, T]],
+      classOf[MapFunction[Row, OUT]],
       body,
       requestedTypeInfo)
-
-    val mapFunction = new MapRunner[Row, T](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
-
-    Some(mapFunction)
   }
-
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
index ff5ffb2..e875587 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
@@ -20,26 +20,26 @@ package org.apache.flink.table.plan.nodes
 
 import org.apache.calcite.plan.{RelOptCost, RelOptPlanner}
 import org.apache.calcite.rex._
-import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
+import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.{CodeGenerator, GeneratedFunction}
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.FlatMapRunner
 import org.apache.flink.types.Row
 
 import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
 
-trait CommonCalc {
+trait CommonCalc[T] {
 
-  private[flink] def functionBody(
+  private[flink] def generateFunction(
       generator: CodeGenerator,
+      ruleDescription: String,
       inputSchema: RowSchema,
       returnSchema: RowSchema,
       calcProgram: RexProgram,
-      config: TableConfig)
-    : String = {
+      config: TableConfig):
+    GeneratedFunction[FlatMapFunction[Row, Row], Row] = {
 
     val expandedExpressions = calcProgram
       .getProjectList
@@ -61,7 +61,7 @@ trait CommonCalc {
       expandedExpressions)
 
     // only projection
-    if (condition == null) {
+    val body = if (condition == null) {
       s"""
         |${projection.code}
         |${generator.collectorTerm}.collect(${projection.resultTerm});
@@ -89,16 +89,12 @@ trait CommonCalc {
           |""".stripMargin
       }
     }
-  }
-
-  private[flink] def calcMapFunction(
-      genFunction: GeneratedFunction[FlatMapFunction[Row, Row], Row])
-    : RichFlatMapFunction[Row, Row] = {
 
-    new FlatMapRunner[Row, Row](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
+    generator.generateFunction(
+      ruleDescription,
+      classOf[FlatMapFunction[Row, Row]],
+      body,
+      returnSchema.physicalTypeInfo)
   }
 
   private[flink] def conditionToString(
@@ -168,8 +164,8 @@ trait CommonCalc {
     // CASTs in RexProgram are reduced as far as possible by ReduceExpressionsRule
     // in normalization stage. So we should ignore CASTs here in optimization stage.
     val compCnt = calcProgram.getExprList.asScala.toList.count {
-      case i: RexInputRef => false
-      case l: RexLiteral => false
+      case _: RexInputRef => false
+      case _: RexLiteral => false
       case c: RexCall if c.getOperator.getName.equals("CAST") => false
       case _ => true
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
index 02305ee..44a109e3 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
@@ -28,7 +28,7 @@ import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NO_CODE}
 import org.apache.flink.table.codegen.{CodeGenerator, GeneratedCollector, GeneratedExpression, GeneratedFunction}
 import org.apache.flink.table.functions.utils.TableSqlFunction
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.{CorrelateFlatMapRunner, TableFunctionCollector}
+import org.apache.flink.table.runtime.TableFunctionCollector
 import org.apache.flink.types.Row
 
 import scala.collection.JavaConverters._
@@ -36,72 +36,27 @@ import scala.collection.JavaConverters._
 /**
   * Join a user-defined table function
   */
-trait CommonCorrelate {
-
-  /**
-    * Creates the [[CorrelateFlatMapRunner]] to execute the join of input table
-    * and user-defined table function.
-    */
-  private[flink] def correlateMapFunction(
-      config: TableConfig,
-      inputSchema: RowSchema,
-      udtfTypeInfo: TypeInformation[Any],
-      returnSchema: RowSchema,
-      joinType: SemiJoinType,
-      rexCall: RexCall,
-      condition: Option[RexNode],
-      pojoFieldMapping: Option[Array[Int]], // udtf return type pojo field mapping
-      ruleDescription: String)
-    : CorrelateFlatMapRunner[Row, Row] = {
-
-    val flatMap = generateFunction(
-      config,
-      inputSchema.physicalTypeInfo,
-      udtfTypeInfo,
-      returnSchema.physicalTypeInfo,
-      returnSchema.logicalFieldNames,
-      joinType,
-      inputSchema.mapRexNode(rexCall).asInstanceOf[RexCall],
-      pojoFieldMapping,
-      ruleDescription)
-
-    val collector = generateCollector(
-      config,
-      inputSchema.physicalTypeInfo,
-      udtfTypeInfo,
-      returnSchema.physicalTypeInfo,
-      returnSchema.logicalFieldNames,
-      condition.map(inputSchema.mapRexNode),
-      pojoFieldMapping)
-
-    new CorrelateFlatMapRunner[Row, Row](
-      flatMap.name,
-      flatMap.code,
-      collector.name,
-      collector.code,
-      flatMap.returnType)
-
-  }
+trait CommonCorrelate[T] {
 
   /**
     * Generates the flat map function to run the user-defined table function.
     */
-  private def generateFunction(
-      config: TableConfig,
-      inputTypeInfo: TypeInformation[Row],
-      udtfTypeInfo: TypeInformation[Any],
-      returnType: TypeInformation[Row],
-      resultFieldNames: Seq[String],
-      joinType: SemiJoinType,
-      rexCall: RexCall,
-      pojoFieldMapping: Option[Array[Int]],
-      ruleDescription: String)
-    : GeneratedFunction[FlatMapFunction[Row, Row], Row] = {
+  private[flink] def generateFunction(
+    config: TableConfig,
+    inputSchema: RowSchema,
+    udtfTypeInfo: TypeInformation[Any],
+    returnSchema: RowSchema,
+    rowType: RelDataType,
+    joinType: SemiJoinType,
+    rexCall: RexCall,
+    pojoFieldMapping: Option[Array[Int]],
+    ruleDescription: String):
+  GeneratedFunction[FlatMapFunction[Row, Row], Row] = {
 
     val functionGenerator = new CodeGenerator(
       config,
       false,
-      inputTypeInfo,
+      inputSchema.physicalTypeInfo,
       Some(udtfTypeInfo),
       None,
       pojoFieldMapping)
@@ -115,9 +70,9 @@ trait CommonCorrelate {
     val call = functionGenerator.generateExpression(rexCall)
     var body =
       s"""
-        |${call.resultTerm}.setCollector($collectorTerm);
-        |${call.code}
-        |""".stripMargin
+         |${call.resultTerm}.setCollector($collectorTerm);
+         |${call.code}
+         |""".stripMargin
 
     if (joinType == SemiJoinType.LEFT) {
       // left outer join
@@ -132,15 +87,17 @@ trait CommonCorrelate {
           x.resultType)
       }
       val outerResultExpr = functionGenerator.generateResultExpression(
-        input1AccessExprs ++ input2NullExprs, returnType, resultFieldNames)
+        input1AccessExprs ++ input2NullExprs,
+        returnSchema.physicalTypeInfo,
+        rowType.getFieldNames.asScala)
       body +=
         s"""
-          |boolean hasOutput = $collectorTerm.isCollected();
-          |if (!hasOutput) {
-          |  ${outerResultExpr.code}
-          |  ${functionGenerator.collectorTerm}.collect(${outerResultExpr.resultTerm});
-          |}
-          |""".stripMargin
+           |boolean hasOutput = $collectorTerm.isCollected();
+           |if (!hasOutput) {
+           |  ${outerResultExpr.code}
+           |  ${functionGenerator.collectorTerm}.collect(${outerResultExpr.resultTerm});
+           |}
+           |""".stripMargin
     } else if (joinType != SemiJoinType.INNER) {
       throw TableException(s"Unsupported SemiJoinType: $joinType for correlate join.")
     }
@@ -149,26 +106,26 @@ trait CommonCorrelate {
       ruleDescription,
       classOf[FlatMapFunction[Row, Row]],
       body,
-      returnType)
+      returnSchema.physicalTypeInfo)
   }
 
   /**
     * Generates table function collector.
     */
   private[flink] def generateCollector(
-      config: TableConfig,
-      inputTypeInfo: TypeInformation[Row],
-      udtfTypeInfo: TypeInformation[Any],
-      returnType: TypeInformation[Row],
-      resultFieldNames: Seq[String],
-      condition: Option[RexNode],
-      pojoFieldMapping: Option[Array[Int]])
-    : GeneratedCollector = {
+    config: TableConfig,
+    inputSchema: RowSchema,
+    udtfTypeInfo: TypeInformation[Any],
+    returnSchema: RowSchema,
+    rowType: RelDataType,
+    condition: Option[RexNode],
+    pojoFieldMapping: Option[Array[Int]])
+  : GeneratedCollector = {
 
     val generator = new CodeGenerator(
       config,
       false,
-      inputTypeInfo,
+      inputSchema.physicalTypeInfo,
       Some(udtfTypeInfo),
       None,
       pojoFieldMapping)
@@ -177,26 +134,26 @@ trait CommonCorrelate {
 
     val crossResultExpr = generator.generateResultExpression(
       input1AccessExprs ++ input2AccessExprs,
-      returnType,
-      resultFieldNames)
+      returnSchema.physicalTypeInfo,
+      rowType.getFieldNames.asScala)
 
     val collectorCode = if (condition.isEmpty) {
       s"""
-        |${crossResultExpr.code}
-        |getCollector().collect(${crossResultExpr.resultTerm});
-        |""".stripMargin
+         |${crossResultExpr.code}
+         |getCollector().collect(${crossResultExpr.resultTerm});
+         |""".stripMargin
     } else {
       val filterGenerator = new CodeGenerator(config, false, udtfTypeInfo, None, pojoFieldMapping)
       filterGenerator.input1Term = filterGenerator.input2Term
       val filterCondition = filterGenerator.generateExpression(condition.get)
       s"""
-        |${filterGenerator.reuseInputUnboxingCode()}
-        |${filterCondition.code}
-        |if (${filterCondition.resultTerm}) {
-        |  ${crossResultExpr.code}
-        |  getCollector().collect(${crossResultExpr.resultTerm});
-        |}
-        |""".stripMargin
+         |${filterGenerator.reuseInputUnboxingCode()}
+         |${filterCondition.code}
+         |if (${filterCondition.resultTerm}) {
+         |  ${crossResultExpr.code}
+         |  getCollector().collect(${crossResultExpr.resultTerm});
+         |}
+         |""".stripMargin
     }
 
     generator.generateTableFunctionCollector(

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
index 091a1ea..7ce73ee 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
@@ -27,7 +27,7 @@ import org.apache.flink.types.Row
 /**
   * Common class for batch and stream scans.
   */
-trait CommonScan {
+trait CommonScan[T] {
 
   /**
     * We check if the input type is exactly the same as the internal row type.
@@ -35,11 +35,8 @@ trait CommonScan {
     */
   private[flink] def needsConversion(
       externalTypeInfo: TypeInformation[Any],
-      internalTypeInfo: TypeInformation[Row])
-    : Boolean = {
-
+      internalTypeInfo: TypeInformation[T]): Boolean =
     externalTypeInfo != internalTypeInfo
-  }
 
   private[flink] def generatedConversionFunction[F <: Function](
       config: TableConfig,

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
index cc5d9fb..95707b8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
@@ -30,7 +30,7 @@ import org.apache.flink.types.Row
 import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
 
-trait BatchScan extends CommonScan with DataSetRel {
+trait BatchScan extends CommonScan[Row] with DataSetRel {
 
   protected def convertToInternalRow(
       input: DataSet[Any],

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
index 5274fa1..e340a8c 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
@@ -24,7 +24,6 @@ import org.apache.calcite.rel.core.Calc
 import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rel.{RelNode, RelWriter}
 import org.apache.calcite.rex._
-import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.api.java.DataSet
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.BatchTableEnvironment
@@ -47,7 +46,7 @@ class DataSetCalc(
     calcProgram: RexProgram,
     ruleDescription: String)
   extends Calc(cluster, traitSet, input, calcProgram)
-  with CommonCalc
+  with CommonCalc[Row]
   with DataSetRel {
 
   override def deriveRowType(): RelDataType = rowRelDataType
@@ -88,22 +87,17 @@ class DataSetCalc(
 
     val generator = new CodeGenerator(config, false, inputDS.getType)
 
-    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+    val returnType = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
 
-    val body = functionBody(
+    val genFunction = generateFunction(
       generator,
+      ruleDescription,
       new RowSchema(getInput.getRowType),
       new RowSchema(getRowType),
       calcProgram,
       config)
 
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatMapFunction[Row, Row]],
-      body,
-      rowTypeInfo)
-
-    val runner = calcMapFunction(genFunction)
+    val runner = new FlatMapRunner(genFunction.name, genFunction.code, returnType)
 
     inputDS.flatMap(runner).name(calcOpName(calcProgram, getExpressionString))
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
index 6c79b45..49ead26 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
@@ -32,6 +32,7 @@ import org.apache.flink.table.functions.utils.TableSqlFunction
 import org.apache.flink.table.plan.nodes.CommonCorrelate
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan
 import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.CorrelateFlatMapRunner
 import org.apache.flink.types.Row
 
 /**
@@ -48,7 +49,7 @@ class DataSetCorrelate(
     joinType: SemiJoinType,
     ruleDescription: String)
   extends SingleRel(cluster, traitSet, inputNode)
-  with CommonCorrelate
+  with CommonCorrelate[Row]
   with DataSetRel {
 
   override def deriveRowType() = relRowType
@@ -98,22 +99,38 @@ class DataSetCorrelate(
     val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]
     val rexCall = funcRel.getCall.asInstanceOf[RexCall]
     val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    val pojoFieldMapping = sqlFunction.getPojoFieldMapping
+    val pojoFieldMapping = Some(sqlFunction.getPojoFieldMapping)
     val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
 
-    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+    val returnType = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
 
-    val mapFunc = correlateMapFunction(
+    val flatMap = generateFunction(
       config,
       new RowSchema(getInput.getRowType),
       udtfTypeInfo,
       new RowSchema(getRowType),
+      rowType,
       joinType,
       rexCall,
-      condition,
-      Some(pojoFieldMapping),
+      pojoFieldMapping,
       ruleDescription)
 
+    val collector = generateCollector(
+      config,
+      new RowSchema(getInput.getRowType),
+      udtfTypeInfo,
+      new RowSchema(getRowType),
+      rowType,
+      condition,
+      pojoFieldMapping)
+
+    val mapFunc = new CorrelateFlatMapRunner[Row, Row](
+      flatMap.name,
+      flatMap.code,
+      collector.name,
+      collector.code,
+      flatMap.returnType)
+
     inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, relRowType))
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
index 3ebee2c..948dd27 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
@@ -88,7 +88,7 @@ class DataSetValues(
       generatedRecords.map(_.code),
       returnType)
 
-    val inputFormat = new ValuesInputFormat[Row](
+    val inputFormat = new ValuesInputFormat(
       generatedFunction.name,
       generatedFunction.code,
       generatedFunction.returnType)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
index c6c25c0..59f723ac 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
@@ -24,13 +24,13 @@ import org.apache.calcite.rel.core.Calc
 import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rel.{RelNode, RelWriter}
 import org.apache.calcite.rex.RexProgram
-import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.nodes.CommonCalc
-import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.types.Row
+import org.apache.flink.table.runtime.CRowFlatMapRunner
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 /**
   * Flink RelNode which matches along with FlatMapOperator.
@@ -45,7 +45,7 @@ class DataStreamCalc(
     calcProgram: RexProgram,
     ruleDescription: String)
   extends Calc(cluster, traitSet, input, calcProgram)
-  with CommonCalc
+  with CommonCalc[CRow]
   with DataStreamRel {
 
   override def deriveRowType(): RelDataType = schema.logicalType
@@ -83,28 +83,28 @@ class DataStreamCalc(
     estimateRowCount(calcProgram, rowCnt)
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
     val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputRowType = inputDataStream.getType.asInstanceOf[CRowTypeInfo].rowType
 
-    val generator = new CodeGenerator(config, false, inputDataStream.getType)
+    val generator = new CodeGenerator(config, false, inputRowType)
 
-    val body = functionBody(
+    val genFunction = generateFunction(
       generator,
+      ruleDescription,
       inputSchema,
       schema,
       calcProgram,
       config)
 
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatMapFunction[Row, Row]],
-      body,
-      schema.physicalTypeInfo)
+    val mapFunc = new CRowFlatMapRunner(
+      genFunction.name,
+      genFunction.code,
+      CRowTypeInfo(schema.physicalTypeInfo))
 
-    val mapFunc = calcMapFunction(genFunction)
     inputDataStream.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
index 899d8ef..19ad89b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
@@ -28,7 +28,8 @@ import org.apache.flink.table.functions.utils.TableSqlFunction
 import org.apache.flink.table.plan.nodes.CommonCorrelate
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.types.Row
+import org.apache.flink.table.runtime.CRowCorrelateFlatMapRunner
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 /**
   * Flink RelNode which matches along with join a user defined table function.
@@ -45,7 +46,7 @@ class DataStreamCorrelate(
     joinType: SemiJoinType,
     ruleDescription: String)
   extends SingleRel(cluster, traitSet, input)
-  with CommonCorrelate
+  with CommonCorrelate[CRow]
   with DataStreamRel {
 
   override def deriveRowType() = schema.logicalType
@@ -81,31 +82,54 @@ class DataStreamCorrelate(
       .itemIf("condition", condition.orNull, condition.isDefined)
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
     // we do not need to specify input type
     val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo]
 
     val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]
     val rexCall = funcRel.getCall.asInstanceOf[RexCall]
     val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    val pojoFieldMapping = sqlFunction.getPojoFieldMapping
+    val pojoFieldMapping = Some(sqlFunction.getPojoFieldMapping)
     val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
 
-    val mapFunc = correlateMapFunction(
+    val flatMap = generateFunction(
       config,
       inputSchema,
       udtfTypeInfo,
       schema,
+      getRowType,
       joinType,
       rexCall,
-      condition,
-      Some(pojoFieldMapping),
+      pojoFieldMapping,
       ruleDescription)
 
-    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, schema.logicalType))
+    val collector = generateCollector(
+      config,
+      inputSchema,
+      udtfTypeInfo,
+      schema,
+      getRowType,
+      condition,
+      pojoFieldMapping)
+
+    val mapFunc = new CRowCorrelateFlatMapRunner(
+      flatMap.name,
+      flatMap.code,
+      collector.name,
+      collector.code,
+      CRowTypeInfo(flatMap.returnType))
+
+    val inputParallelism = inputDS.getParallelism
+
+    inputDS
+      .flatMap(mapFunc)
+      // preserve input parallelism to ensure that acc and retract messages remain in order
+      .setParallelism(inputParallelism)
+      .name(correlateOpName(rexCall, sqlFunction, schema.logicalType))
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
index 3555c80..056cda9 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
@@ -28,8 +28,9 @@ import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.runtime.aggregate._
 import org.apache.flink.table.plan.nodes.CommonAggregate
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.types.Row
+import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules
 import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 /**
   *
@@ -97,15 +98,18 @@ class DataStreamGroupAggregate(
         inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil))
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
 
     val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+
     val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
       new CalcitePair[AggregateCall, String](
         inputSchema.mapAggregateCall(namedAggregate.left),
         namedAggregate.right)
     }
 
+    val outRowType = CRowTypeInfo(schema.physicalTypeInfo)
+
     val generator = new CodeGenerator(
       tableEnv.getConfig,
       false,
@@ -129,28 +133,30 @@ class DataStreamGroupAggregate(
       physicalNamedAggregates,
       inputSchema.logicalType,
       inputSchema.physicalFieldTypeInfo,
-      groupings)
+      groupings,
+      DataStreamRetractionRules.isAccRetract(this),
+      DataStreamRetractionRules.isAccRetract(getInput))
 
-    val result: DataStream[Row] =
+    val result: DataStream[CRow] =
     // grouped / keyed aggregation
       if (physicalGrouping.nonEmpty) {
         inputDS
         .keyBy(groupings: _*)
         .process(processFunction)
-        .returns(schema.physicalTypeInfo)
+        .returns(outRowType)
         .name(keyedAggOpName)
-        .asInstanceOf[DataStream[Row]]
+        .asInstanceOf[DataStream[CRow]]
       }
       // global / non-keyed aggregation
       else {
         inputDS
-        .keyBy(new NullByteKeySelector[Row])
+        .keyBy(new NullByteKeySelector[CRow])
         .process(processFunction)
         .setParallelism(1)
         .setMaxParallelism(1)
-        .returns(schema.physicalTypeInfo)
+        .returns(outRowType)
         .name(nonKeyedAggOpName)
-        .asInstanceOf[DataStream[Row]]
+        .asInstanceOf[DataStream[CRow]]
       }
     result
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
index ea4b0bf..f61828b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
@@ -26,7 +26,7 @@ import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream}
 import org.apache.flink.streaming.api.windowing.assigners._
 import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
-import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
 import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.expressions.ExpressionUtils._
@@ -34,10 +34,12 @@ import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.plan.nodes.CommonAggregate
 import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.plan.nodes.datastream.DataStreamGroupWindowAggregate._
+import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules
 import org.apache.flink.table.runtime.aggregate.AggregateUtil._
 import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
-import org.apache.flink.types.Row
+import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
 
 class DataStreamGroupWindowAggregate(
     window: LogicalWindow,
@@ -101,14 +103,25 @@ class DataStreamGroupWindowAggregate(
           namedProperties))
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
 
     val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+
     val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
       new CalcitePair[AggregateCall, String](
         inputSchema.mapAggregateCall(namedAggregate.left),
         namedAggregate.right)
     }
+    val consumeRetraction = DataStreamRetractionRules.isAccRetract(input)
+
+    if (consumeRetraction) {
+      throw new TableException(
+        "Retraction on windowed GroupBy aggregation is not supported yet. " +
+          "Note: Windowed GroupBy aggregation should not follow a " +
+          "non-windowed GroupBy aggregation.")
+    }
+
+    val outRowType = CRowTypeInfo(schema.physicalTypeInfo)
 
     val aggString = aggregationToString(
       inputSchema.logicalType,
@@ -145,7 +158,7 @@ class DataStreamGroupWindowAggregate(
       val keyedStream = inputDS.keyBy(physicalGrouping: _*)
       val windowedStream =
         createKeyedWindowedStream(window, keyedStream)
-          .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
+          .asInstanceOf[WindowedStream[CRow, Tuple, DataStreamWindow]]
 
       val (aggFunction, accumulatorRowType, aggResultRowType) =
         AggregateUtil.createDataStreamAggregateFunction(
@@ -154,15 +167,11 @@ class DataStreamGroupWindowAggregate(
           inputSchema.physicalType,
           inputSchema.physicalFieldTypeInfo,
           schema.physicalType,
+          physicalGrouping,
           needMerge)
 
       windowedStream
-        .aggregate(
-          aggFunction,
-          windowFunction,
-          accumulatorRowType,
-          aggResultRowType,
-          schema.physicalTypeInfo)
+        .aggregate(aggFunction, windowFunction, accumulatorRowType, aggResultRowType, outRowType)
         .name(keyedAggOpName)
     }
     // global / non-keyed aggregation
@@ -174,7 +183,7 @@ class DataStreamGroupWindowAggregate(
 
       val windowedStream =
         createNonKeyedWindowedStream(window, inputDS)
-          .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
+          .asInstanceOf[AllWindowedStream[CRow, DataStreamWindow]]
 
       val (aggFunction, accumulatorRowType, aggResultRowType) =
         AggregateUtil.createDataStreamAggregateFunction(
@@ -183,15 +192,11 @@ class DataStreamGroupWindowAggregate(
           inputSchema.physicalType,
           inputSchema.physicalFieldTypeInfo,
           schema.physicalType,
+          Array[Int](),
           needMerge)
 
       windowedStream
-        .aggregate(
-          aggFunction,
-          windowFunction,
-          accumulatorRowType,
-          aggResultRowType,
-          schema.physicalTypeInfo)
+        .aggregate(aggFunction, windowFunction, accumulatorRowType, aggResultRowType, outRowType)
         .name(nonKeyedAggOpName)
     }
   }
@@ -199,9 +204,10 @@ class DataStreamGroupWindowAggregate(
 
 object DataStreamGroupWindowAggregate {
 
-
-  private def createKeyedWindowedStream(groupWindow: LogicalWindow, stream: KeyedStream[Row, Tuple])
-    : WindowedStream[Row, Tuple, _ <: DataStreamWindow] = groupWindow match {
+  private def createKeyedWindowedStream(
+      groupWindow: LogicalWindow,
+      stream: KeyedStream[CRow, Tuple]):
+    WindowedStream[CRow, Tuple, _ <: DataStreamWindow] = groupWindow match {
 
     case TumblingGroupWindow(_, timeField, size)
         if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size)=>
@@ -250,8 +256,10 @@ object DataStreamGroupWindowAggregate {
       stream.window(EventTimeSessionWindows.withGap(toTime(gap)))
   }
 
-  private def createNonKeyedWindowedStream(groupWindow: LogicalWindow, stream: DataStream[Row])
-    : AllWindowedStream[Row, _ <: DataStreamWindow] = groupWindow match {
+  private def createNonKeyedWindowedStream(
+      groupWindow: LogicalWindow,
+      stream: DataStream[CRow]):
+    AllWindowedStream[CRow, _ <: DataStreamWindow] = groupWindow match {
 
     case TumblingGroupWindow(_, timeField, size)
         if isProctimeAttribute(timeField) && isTimeIntervalLiteral(size) =>

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index fb912c4..e823cd6 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -20,21 +20,22 @@ package org.apache.flink.table.plan.nodes.datastream
 import java.util.{List => JList}
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.Window.Group
 import org.apache.calcite.rel.core.{AggregateCall, Window}
 import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
-import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.nodes.OverAggregate
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
 import org.apache.flink.table.runtime.aggregate._
-import org.apache.flink.types.Row
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules
+import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 class DataStreamOverAggregate(
     logicWindow: Window,
@@ -87,7 +88,7 @@ class DataStreamOverAggregate(
           namedAggregates))
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
     if (logicWindow.groups.size > 1) {
       throw new TableException(
         "Unsupported use of OVER windows. All aggregates must be computed on the same window.")
@@ -110,6 +111,8 @@ class DataStreamOverAggregate(
 
     val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
 
+    val consumeRetraction = DataStreamRetractionRules.isAccRetract(input)
+
     val generator = new CodeGenerator(
       tableEnv.getConfig,
       false,
@@ -120,6 +123,12 @@ class DataStreamOverAggregate(
       .get(orderKey.getFieldIndex)
       .getType
 
+    if (consumeRetraction) {
+      throw new TableException(
+        "Retraction on Over window aggregation is not supported yet. " +
+          "Note: Over window aggregation should not follow a non-windowed GroupBy aggregation.")
+    }
+
     timeType match {
       case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
         // proc-time OVER window
@@ -138,8 +147,7 @@ class DataStreamOverAggregate(
             generator,
             inputDS,
             isRowTimeType = false,
-            isRowsClause = overWindow.isRows
-          )
+            isRowsClause = overWindow.isRows)
         } else {
           throw new TableException(
             "OVER RANGE FOLLOWING windows are not supported yet.")
@@ -154,16 +162,14 @@ class DataStreamOverAggregate(
             generator,
             inputDS,
             isRowTimeType = true,
-            isRowsClause = overWindow.isRows
-          )
+            isRowsClause = overWindow.isRows)
         } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) {
           // bounded OVER window
           createBoundedAndCurrentRowOverWindow(
             generator,
             inputDS,
             isRowTimeType = true,
-            isRowsClause = overWindow.isRows
-          )
+            isRowsClause = overWindow.isRows)
         } else {
           throw new TableException(
             "OVER RANGE FOLLOWING windows are not supported yet.")
@@ -177,12 +183,14 @@ class DataStreamOverAggregate(
 
   def createUnboundedAndCurrentRowOverWindow(
     generator: CodeGenerator,
-    inputDS: DataStream[Row],
+    inputDS: DataStream[CRow],
     isRowTimeType: Boolean,
-    isRowsClause: Boolean): DataStream[Row] = {
+    isRowsClause: Boolean): DataStream[CRow] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
+
     val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex)
+
     val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates.map {
       namedAggregate =>
         new CalcitePair[AggregateCall, String](
@@ -190,6 +198,9 @@ class DataStreamOverAggregate(
           namedAggregate.right)
     }
 
+    // get the output types
+    val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
+
     val processFunction = AggregateUtil.createUnboundedOverProcessFunction(
       generator,
       namedAggregates,
@@ -200,30 +211,28 @@ class DataStreamOverAggregate(
       partitionKeys.nonEmpty,
       isRowsClause)
 
-    val result: DataStream[Row] =
+    val result: DataStream[CRow] =
     // partitioned aggregation
       if (partitionKeys.nonEmpty) {
         inputDS
           .keyBy(partitionKeys: _*)
           .process(processFunction)
-          .returns(schema.physicalTypeInfo)
+          .returns(returnTypeInfo)
           .name(aggOpName)
-          .asInstanceOf[DataStream[Row]]
+          .asInstanceOf[DataStream[CRow]]
       }
       // non-partitioned aggregation
       else {
         if (isRowTimeType) {
-          inputDS.keyBy(new NullByteKeySelector[Row])
+          inputDS.keyBy(new NullByteKeySelector[CRow])
             .process(processFunction).setParallelism(1).setMaxParallelism(1)
-            .returns(schema.physicalTypeInfo)
+            .returns(returnTypeInfo)
             .name(aggOpName)
-            .asInstanceOf[DataStream[Row]]
         } else {
           inputDS
             .process(processFunction).setParallelism(1).setMaxParallelism(1)
-            .returns(schema.physicalTypeInfo)
+            .returns(returnTypeInfo)
             .name(aggOpName)
-            .asInstanceOf[DataStream[Row]]
         }
       }
     result
@@ -231,9 +240,9 @@ class DataStreamOverAggregate(
 
   def createBoundedAndCurrentRowOverWindow(
     generator: CodeGenerator,
-    inputDS: DataStream[Row],
+    inputDS: DataStream[CRow],
     isRowTimeType: Boolean,
-    isRowsClause: Boolean): DataStream[Row] = {
+    isRowsClause: Boolean): DataStream[CRow] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
     val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex)
@@ -245,7 +254,10 @@ class DataStreamOverAggregate(
     }
 
     val precedingOffset =
-      getLowerBoundary(logicWindow, overWindow, input) + (if (isRowsClause) 1 else 0)
+      getLowerBoundary(logicWindow, overWindow, getInput()) + (if (isRowsClause) 1 else 0)
+
+    // get the output types
+    val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
 
     val processFunction = AggregateUtil.createBoundedOverProcessFunction(
       generator,
@@ -257,24 +269,22 @@ class DataStreamOverAggregate(
       isRowsClause,
       isRowTimeType
     )
-    val result: DataStream[Row] =
+    val result: DataStream[CRow] =
     // partitioned aggregation
       if (partitionKeys.nonEmpty) {
         inputDS
           .keyBy(partitionKeys: _*)
           .process(processFunction)
-          .returns(schema.physicalTypeInfo)
+          .returns(returnTypeInfo)
           .name(aggOpName)
-          .asInstanceOf[DataStream[Row]]
       }
       // non-partitioned aggregation
       else {
         inputDS
-          .keyBy(new NullByteKeySelector[Row])
+          .keyBy(new NullByteKeySelector[CRow])
           .process(processFunction).setParallelism(1).setMaxParallelism(1)
-          .returns(schema.physicalTypeInfo)
+          .returns(returnTypeInfo)
           .name(aggOpName)
-          .asInstanceOf[DataStream[Row]]
       }
     result
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
index 128da81..9754de4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.nodes.datastream
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
 import org.apache.flink.table.plan.nodes.FlinkRelNode
-import org.apache.flink.types.Row
+import org.apache.flink.table.runtime.types.CRow
 
 trait DataStreamRel extends FlinkRelNode {
 
@@ -29,9 +29,9 @@ trait DataStreamRel extends FlinkRelNode {
     * Translates the FlinkRelNode into a Flink operator.
     *
     * @param tableEnv The [[StreamTableEnvironment]] of the translated Table.
-    * @return DataStream of type [[Row]]
+    * @return DataStream of type [[CRow]]
     */
-  def translateToPlan(tableEnv: StreamTableEnvironment) : DataStream[Row]
+  def translateToPlan(tableEnv: StreamTableEnvironment) : DataStream[CRow]
 
   /**
     * Whether the [[DataStreamRel]] requires that update and delete changes are sent with retraction

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
index 05f60ba..c613646 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
@@ -24,8 +24,9 @@ import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.TableScan
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
-import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema}
-import org.apache.flink.types.Row
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.table.runtime.types.CRow
 
 /**
   * Flink RelNode which matches along with DataStreamSource.
@@ -53,7 +54,7 @@ class DataStreamScan(
     )
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
     val config = tableEnv.getConfig
     val inputDataStream: DataStream[Any] = dataStreamTable.dataStream
     convertToInternalRow(schema, inputDataStream, dataStreamTable, config)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
index 47b4946..654c259 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
@@ -23,7 +23,7 @@ import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.types.Row
+import org.apache.flink.table.runtime.types.CRow
 
 /**
   * Flink RelNode which matches along with Union.
@@ -58,7 +58,7 @@ class DataStreamUnion(
     s"Union All(union: (${schema.logicalFieldNames.mkString(", ")}))"
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
 
     val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
     val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
index c964e03..32c9aaf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
@@ -27,8 +27,8 @@ import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.io.ValuesInputFormat
-import org.apache.flink.types.Row
+import org.apache.flink.table.runtime.io.CRowValuesInputFormat
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 import scala.collection.JavaConverters._
 
@@ -56,10 +56,11 @@ class DataStreamValues(
     )
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
+    val returnType = CRowTypeInfo(schema.physicalTypeInfo)
     val generator = new CodeGenerator(config)
 
     // generate code for every record
@@ -76,12 +77,12 @@ class DataStreamValues(
       generatedRecords.map(_.code),
       schema.physicalTypeInfo)
 
-    val inputFormat = new ValuesInputFormat[Row](
+    val inputFormat = new CRowValuesInputFormat(
       generatedFunction.name,
       generatedFunction.code,
-      generatedFunction.returnType)
+      returnType)
 
-    tableEnv.execEnv.createInput(inputFormat, schema.physicalTypeInfo)
+    tableEnv.execEnv.createInput(inputFormat, returnType)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
index dd82819..25e72fa 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
@@ -22,46 +22,51 @@ import org.apache.flink.api.common.functions.MapFunction
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.plan.nodes.CommonScan
-import org.apache.flink.table.plan.schema.{FlinkTable, RowSchema}
-import org.apache.flink.table.runtime.MapRunner
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.types.Row
+import org.apache.flink.table.plan.schema.FlinkTable
+import org.apache.flink.table.runtime.CRowOutputMapRunner
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 import scala.collection.JavaConverters._
 
-trait StreamScan extends CommonScan with DataStreamRel {
+trait StreamScan extends CommonScan[CRow] with DataStreamRel {
 
   protected def convertToInternalRow(
       schema: RowSchema,
       input: DataStream[Any],
       flinkTable: FlinkTable[_],
       config: TableConfig)
-    : DataStream[Row] = {
+    : DataStream[CRow] = {
+
+    val inputType = input.getType
+    val internalType = CRowTypeInfo(schema.physicalTypeInfo)
 
     // conversion
-    if (needsConversion(input.getType, schema.physicalTypeInfo)) {
+    if (needsConversion(input.getType, internalType)) {
 
       val function = generatedConversionFunction(
         config,
         classOf[MapFunction[Any, Row]],
-        input.getType,
+        inputType,
         schema.physicalTypeInfo,
         "DataStreamSourceConversion",
         schema.physicalFieldNames,
         Some(flinkTable.fieldIndexes))
 
-      val runner = new MapRunner[Any, Row](
+      val mapFunc = new CRowOutputMapRunner(
         function.name,
         function.code,
-        function.returnType)
+        internalType)
 
-      val opName = s"from: (${schema.logicalFieldNames.mkString(", ")})"
+      val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
 
       // TODO we need a ProcessFunction here
-      input.map(runner).name(opName)
+      input.map(mapFunc).name(opName).returns(internalType)
     }
     // no conversion necessary, forward
     else {
-      input.asInstanceOf[DataStream[Row]]
+      input.asInstanceOf[DataStream[CRow]]
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
index e34e416..b2d7019 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
@@ -25,9 +25,11 @@ import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.PhysicalTableSourceScan
-import org.apache.flink.table.plan.schema.{RowSchema, TableSourceTable}
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.sources._
-import org.apache.flink.types.Row
+import org.apache.flink.table.plan.schema.TableSourceTable
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.sources.{StreamTableSource, TableSource}
 
 /** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */
 class StreamTableSourceScan(
@@ -96,7 +98,7 @@ class StreamTableSourceScan(
     )
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
+  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
     val config = tableEnv.getConfig
     val inputDataStream = tableSource.getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]]
     convertToInternalRow(

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
index ec90392..0ca079e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
@@ -34,7 +34,7 @@ class FlinkLogicalCalc(
     calcProgram: RexProgram)
   extends Calc(cluster, traitSet, input, calcProgram)
   with FlinkLogicalRel
-  with CommonCalc {
+  with CommonCalc[Any] {
 
   override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = {
     new FlinkLogicalCalc(cluster, traitSet, child, program)

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
index aeb67b6..bd9a7ee 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
@@ -82,6 +82,14 @@ object DataStreamRetractionRules {
   }
 
   /**
+    * Checks if a [[RelNode]] is in [[AccMode.AccRetract]] mode.
+    */
+  def isAccRetract(node: RelNode): Boolean = {
+    val accModeTrait = node.getTraitSet.getTrait(AccModeTraitDef.INSTANCE)
+    null != accModeTrait && accModeTrait.getAccMode == AccMode.AccRetract
+  }
+
+  /**
     * Rule that assigns the default retraction information to [[DataStreamRel]] nodes.
     * The default is to not publish updates as retraction messages and [[AccMode.Acc]].
     */
@@ -190,14 +198,6 @@ object DataStreamRetractionRules {
     }
 
     /**
-      * Checks if a [[RelNode]] is in [[AccMode.AccRetract]] mode.
-      */
-    def isAccRetract(node: RelNode): Boolean = {
-      val accModeTrait = node.getTraitSet.getTrait(AccModeTraitDef.INSTANCE)
-      null != accModeTrait && accModeTrait.getAccMode == AccMode.AccRetract
-    }
-
-    /**
       * Set [[AccMode.AccRetract]] to a [[RelNode]].
       */
     def setAccRetract(relNode: RelNode): RelNode = {

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
new file mode 100644
index 0000000..66e51b1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
@@ -0,0 +1,83 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
+import org.apache.flink.api.common.functions.util.FunctionUtils
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.{Logger, LoggerFactory}
+
+/**
+  * A CorrelateFlatMapRunner with [[CRow]] input and [[CRow]] output.
+  */
+class CRowCorrelateFlatMapRunner(
+    flatMapName: String,
+    flatMapCode: String,
+    collectorName: String,
+    collectorCode: String,
+    @transient returnType: TypeInformation[CRow])
+  extends RichFlatMapFunction[CRow, CRow]
+  with ResultTypeQueryable[CRow]
+  with Compiler[Any] {
+
+  val LOG: Logger = LoggerFactory.getLogger(this.getClass)
+
+  private var function: FlatMapFunction[Row, Row] = _
+  private var collector: TableFunctionCollector[_] = _
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling TableFunctionCollector: $collectorName \n\n Code:\n$collectorCode")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, collectorName, collectorCode)
+    LOG.debug("Instantiating TableFunctionCollector.")
+    collector = clazz.newInstance().asInstanceOf[TableFunctionCollector[_]]
+    this.cRowWrapper = new CRowWrappingCollector()
+
+    LOG.debug(s"Compiling FlatMapFunction: $flatMapName \n\n Code:\n$flatMapCode")
+    val flatMapClazz = compile(getRuntimeContext.getUserCodeClassLoader, flatMapName, flatMapCode)
+    val constructor = flatMapClazz.getConstructor(classOf[TableFunctionCollector[_]])
+    LOG.debug("Instantiating FlatMapFunction.")
+    function = constructor.newInstance(collector).asInstanceOf[FlatMapFunction[Row, Row]]
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
+  }
+
+  override def flatMap(in: CRow, out: Collector[CRow]): Unit = {
+    cRowWrapper.out = out
+    cRowWrapper.setChange(in.change)
+
+    collector.setCollector(cRowWrapper)
+    collector.setInput(in.row)
+    collector.reset()
+
+    function.flatMap(in.row, cRowWrapper)
+  }
+
+  override def getProducedType: TypeInformation[CRow] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/27bf4cab/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
new file mode 100644
index 0000000..9a4650b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.table.runtime
+
+import org.apache.flink.api.common.functions.util.FunctionUtils
+import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * FlatMapRunner with [[CRow]] input and [[CRow]] output.
+  */
+class CRowFlatMapRunner(
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[CRow])
+  extends RichFlatMapFunction[CRow, CRow]
+  with ResultTypeQueryable[CRow]
+  with Compiler[FlatMapFunction[Row, Row]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: FlatMapFunction[Row, Row] = _
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling FlatMapFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating FlatMapFunction.")
+    function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
+
+    this.cRowWrapper = new CRowWrappingCollector()
+  }
+
+  override def flatMap(in: CRow, out: Collector[CRow]): Unit = {
+    cRowWrapper.out = out
+    cRowWrapper.setChange(in.change)
+    function.flatMap(in.row, cRowWrapper)
+  }
+
+  override def getProducedType: TypeInformation[CRow] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
+}
+
+


[06/15] flink git commit: [FLINK-6090] [table] Add RetractionRules for annotating AccMode to DataStreamRel nodes.

Posted by fh...@apache.org.
[FLINK-6090] [table] Add RetractionRules for annotating AccMode to DataStreamRel nodes.


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

Branch: refs/heads/master
Commit: dc54abc694c3695270695be2e5bc59a7f91ee460
Parents: 8f78824
Author: hequn.chq <he...@alibaba-inc.com>
Authored: Fri Apr 7 13:12:04 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat May 6 01:51:54 2017 +0200

----------------------------------------------------------------------
 .../nodes/datastream/DataStreamCorrelate.scala  |   6 +-
 .../datastream/DataStreamGroupAggregate.scala   |   6 +
 .../DataStreamGroupWindowAggregate.scala        |   4 +
 .../datastream/DataStreamOverAggregate.scala    |   6 +-
 .../plan/nodes/datastream/DataStreamRel.scala   |  17 +
 .../nodes/datastream/retractionTraitDefs.scala  |  81 +++++
 .../nodes/datastream/retractionTraits.scala     | 100 ++++++
 .../flink/table/plan/rules/FlinkRuleSets.scala  |   7 +-
 .../datastream/DataStreamRetractionRules.scala  | 248 ++++++++++++++
 .../flink/table/CalciteConfigBuilderTest.scala  |  39 +--
 .../table/plan/rules/RetractionRulesTest.scala  | 321 +++++++++++++++++++
 11 files changed, 810 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
index 8955110..899d8ef 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
@@ -37,14 +37,14 @@ class DataStreamCorrelate(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
     inputSchema: RowSchema,
-    inputNode: RelNode,
+    input: RelNode,
     scan: FlinkLogicalTableFunctionScan,
     condition: Option[RexNode],
     schema: RowSchema,
     joinSchema: RowSchema,
     joinType: SemiJoinType,
     ruleDescription: String)
-  extends SingleRel(cluster, traitSet, inputNode)
+  extends SingleRel(cluster, traitSet, input)
   with CommonCorrelate
   with DataStreamRel {
 
@@ -86,7 +86,7 @@ class DataStreamCorrelate(
     val config = tableEnv.getConfig
 
     // we do not need to specify input type
-    val inputDS = inputNode.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
 
     val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]
     val rexCall = funcRel.getCall.asInstanceOf[RexCall]

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
index 19f90c7..3555c80 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
@@ -60,6 +60,12 @@ class DataStreamGroupAggregate(
 
   override def deriveRowType() = schema.logicalType
 
+  override def needsUpdatesAsRetraction = true
+
+  override def producesUpdates = true
+
+  override def consumesRetractions = true
+
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamGroupAggregate(
       cluster,

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
index 5aced66..ea4b0bf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
@@ -53,6 +53,10 @@ class DataStreamGroupWindowAggregate(
 
   override def deriveRowType(): RelDataType = schema.logicalType
 
+  override def needsUpdatesAsRetraction = true
+
+  override def consumesRetractions = true
+
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamGroupWindowAggregate(
       window,

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index db31f32..fb912c4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -49,6 +49,10 @@ class DataStreamOverAggregate(
 
   override def deriveRowType(): RelDataType = schema.logicalType
 
+  override def needsUpdatesAsRetraction = true
+
+  override def consumesRetractions = true
+
   override def copy(traitSet: RelTraitSet, inputs: JList[RelNode]): RelNode = {
     new DataStreamOverAggregate(
       logicWindow,
@@ -297,7 +301,7 @@ class DataStreamOverAggregate(
     }ORDER BY: ${orderingToString(inputSchema.logicalType,
         overWindow.orderKeys.getFieldCollations)}, " +
       s"${if (overWindow.isRows) "ROWS" else "RANGE"}" +
-      s"${windowRange(logicWindow, overWindow, inputNode.asInstanceOf[DataStreamRel])}, " +
+      s"${windowRange(logicWindow, overWindow, inputNode)}, " +
       s"select: (${
         aggregationToString(
           inputSchema.logicalType,

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
index 03938f3..128da81 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
@@ -33,4 +33,21 @@ trait DataStreamRel extends FlinkRelNode {
     */
   def translateToPlan(tableEnv: StreamTableEnvironment) : DataStream[Row]
 
+  /**
+    * Whether the [[DataStreamRel]] requires that update and delete changes are sent with retraction
+    * messages.
+    */
+  def needsUpdatesAsRetraction: Boolean = false
+
+  /**
+    * Whether the [[DataStreamRel]] produces update and delete changes.
+    */
+  def producesUpdates: Boolean = false
+
+  /**
+    * Wheter the [[DataStreamRel]] consumes retraction messages instead of forwarding them.
+    * The node might or might not produce new retraction messages.
+    */
+  def consumesRetractions: Boolean = false
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraitDefs.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraitDefs.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraitDefs.scala
new file mode 100644
index 0000000..c43d951
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraitDefs.scala
@@ -0,0 +1,81 @@
+/*
+ * 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.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptPlanner, RelTraitDef}
+import org.apache.calcite.rel.RelNode
+
+/**
+  * Definition of the [[UpdateAsRetractionTrait]].
+  */
+class UpdateAsRetractionTraitDef extends RelTraitDef[UpdateAsRetractionTrait] {
+  override def convert(
+      planner: RelOptPlanner,
+      rel: RelNode,
+      toTrait: UpdateAsRetractionTrait,
+      allowInfiniteCostConverters: Boolean): RelNode = {
+
+    rel.copy(rel.getTraitSet.plus(toTrait), rel.getInputs)
+  }
+
+  override def canConvert(
+      planner: RelOptPlanner,
+      fromTrait: UpdateAsRetractionTrait,
+      toTrait: UpdateAsRetractionTrait): Boolean = true
+
+  override def getTraitClass: Class[UpdateAsRetractionTrait] = classOf[UpdateAsRetractionTrait]
+
+  override def getSimpleName: String = this.getClass.getSimpleName
+
+  override def getDefault: UpdateAsRetractionTrait = UpdateAsRetractionTrait.DEFAULT
+}
+
+object UpdateAsRetractionTraitDef {
+  val INSTANCE = new UpdateAsRetractionTraitDef
+}
+
+/**
+  * Definition of the [[AccModeTrait]].
+  */
+class AccModeTraitDef extends RelTraitDef[AccModeTrait] {
+
+  override def convert(
+      planner: RelOptPlanner,
+      rel: RelNode,
+      toTrait: AccModeTrait,
+      allowInfiniteCostConverters: Boolean): RelNode = {
+
+    rel.copy(rel.getTraitSet.plus(toTrait), rel.getInputs)
+  }
+
+  override def canConvert(
+      planner: RelOptPlanner,
+      fromTrait: AccModeTrait,
+      toTrait: AccModeTrait): Boolean = true
+
+  override def getTraitClass: Class[AccModeTrait] = classOf[AccModeTrait]
+
+  override def getSimpleName: String = this.getClass.getSimpleName
+
+  override def getDefault: AccModeTrait = AccModeTrait.DEFAULT
+}
+
+object AccModeTraitDef {
+  val INSTANCE = new AccModeTraitDef
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
new file mode 100644
index 0000000..c3b43ba
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/retractionTraits.scala
@@ -0,0 +1,100 @@
+/*
+ * 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.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptPlanner, RelTrait, RelTraitDef}
+import org.apache.flink.table.plan.nodes.datastream.AccMode.AccMode
+
+/** Tracks if a [[org.apache.calcite.rel.RelNode]] needs to send update and delete changes as
+  * retraction messages.
+  */
+class UpdateAsRetractionTrait extends RelTrait {
+
+  /**
+    * Defines whether the [[org.apache.calcite.rel.RelNode]] needs to send update and delete
+    * changes as retraction messages.
+    */
+  private var updateAsRetraction: Boolean = false
+
+  def this(updateAsRetraction: Boolean) {
+    this()
+    this.updateAsRetraction = updateAsRetraction
+  }
+
+  def sendsUpdatesAsRetractions: Boolean = updateAsRetraction
+
+  override def register(planner: RelOptPlanner): Unit = { }
+
+  override def getTraitDef: RelTraitDef[_ <: RelTrait] = UpdateAsRetractionTraitDef.INSTANCE
+
+  override def satisfies(`trait`: RelTrait): Boolean = this.equals(`trait`)
+
+  override def toString: String = updateAsRetraction.toString
+
+}
+
+object UpdateAsRetractionTrait {
+  val DEFAULT = new UpdateAsRetractionTrait(false)
+}
+
+/**
+  * Tracks the AccMode of a [[org.apache.calcite.rel.RelNode]].
+  */
+class AccModeTrait extends RelTrait {
+
+  /** Defines the accumulating mode for a operator. */
+  private var accMode = AccMode.Acc
+
+  def this(accMode: AccMode) {
+    this()
+    this.accMode = accMode
+  }
+
+  def getAccMode: AccMode = accMode
+
+  override def register(planner: RelOptPlanner): Unit = { }
+
+  override def getTraitDef: RelTraitDef[_ <: RelTrait] = AccModeTraitDef.INSTANCE
+
+  override def satisfies(`trait`: RelTrait): Boolean = this.equals(`trait`)
+
+  override def toString: String = accMode.toString
+}
+
+object AccModeTrait {
+  val DEFAULT = new AccModeTrait(AccMode.Acc)
+}
+
+/**
+  * The AccMode indicates which kinds of messages a [[org.apache.calcite.rel.RelNode]] might
+  * produce.
+  * In [[AccMode.Acc]] the node only emit accumulate messages.
+  * In [[AccMode.AccRetract]], the node produces accumulate messages for insert changes,
+  * retraction messages for delete changes, and accumulate and retraction messages
+  * for update changes.
+  */
+object AccMode extends Enumeration {
+  type AccMode = Value
+
+  val Acc        = Value // Operator produces only accumulate (insert) messages
+  val AccRetract = Value // Operator produces accumulate (insert, update) and
+                         //   retraction (delete, update) messages
+}
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
index f4de651..fad60fd 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
@@ -21,9 +21,9 @@ package org.apache.flink.table.plan.rules
 import org.apache.calcite.rel.rules._
 import org.apache.calcite.tools.{RuleSet, RuleSets}
 import org.apache.flink.table.plan.rules.common._
+import org.apache.flink.table.plan.rules.logical._
 import org.apache.flink.table.plan.rules.dataSet._
 import org.apache.flink.table.plan.rules.datastream._
-import org.apache.flink.table.plan.rules.logical._
 import org.apache.flink.table.plan.nodes.logical._
 
 object FlinkRuleSets {
@@ -191,7 +191,10 @@ object FlinkRuleSets {
     * RuleSet to decorate plans for stream / DataStream execution
     */
   val DATASTREAM_DECO_RULES: RuleSet = RuleSets.ofList(
-    // rules
+    // retraction rules
+    DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE,
+    DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE,
+    DataStreamRetractionRules.ACCMODE_INSTANCE
   )
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
new file mode 100644
index 0000000..aeb67b6
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamRetractionRules.scala
@@ -0,0 +1,248 @@
+/*
+ * 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.table.plan.rules.datastream
+
+import org.apache.calcite.plan.RelOptRule.{operand, _}
+import org.apache.calcite.plan.hep.HepRelVertex
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.RelNode
+import org.apache.flink.table.plan.nodes.datastream._
+
+import scala.collection.JavaConverters._
+
+/**
+  * Collection of rules to annotate [[DataStreamRel]] nodes with retraction information.
+  *
+  * The rules have to be applied in the following order:
+  * - [[DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE]]
+  * - [[DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE]]
+  * - [[DataStreamRetractionRules.ACCMODE_INSTANCE]]
+  *
+  * The rules will assign a [[AccModeTrait]] to each [[DataStreamRel]] node of the plan. The
+  * trait defines the [[AccMode]] a node.
+  * - [[AccMode.Acc]] defines that the node produces only accumulate messages, i.e., all types of
+  * modifications (insert, update, delete) are encoded as accumulate messages.
+  * - [[AccMode.AccRetract]] defines that the node produces accumulate and retraction messages.
+  * Insert modifications are encoded as accumulate message, delete modifications as retraction
+  * message, and update modifications as a pair of accumulate and retraction messages.
+  *
+  */
+object DataStreamRetractionRules {
+
+  /**
+    * Rule instance that assigns default retraction to [[DataStreamRel]] nodes.
+    */
+  val DEFAULT_RETRACTION_INSTANCE = new AssignDefaultRetractionRule()
+
+  /**
+    * Rule instance that checks if [[DataStreamRel]] nodes need to ship updates as retractions.
+    */
+  val UPDATES_AS_RETRACTION_INSTANCE = new SetUpdatesAsRetractionRule()
+
+  /**
+    * Rule instance that assigns the [[AccMode]] to [[DataStreamRel]] nodes.
+    */
+  val ACCMODE_INSTANCE = new SetAccModeRule()
+
+  /**
+    * Get all children RelNodes of a RelNode.
+    *
+    * @param parent The parent RelNode
+    * @return All child nodes
+    */
+  def getChildRelNodes(parent: RelNode): Seq[RelNode] = {
+    parent.getInputs.asScala.map(_.asInstanceOf[HepRelVertex].getCurrentRel)
+  }
+
+  /**
+    * Checks if a [[RelNode]] ships updates as retractions.
+    *
+    * @param node The node to check.
+    * @return True if the node ships updates as retractions, false otherwise.
+    */
+  def sendsUpdatesAsRetraction(node: RelNode): Boolean = {
+    val retractionTrait = node.getTraitSet.getTrait(UpdateAsRetractionTraitDef.INSTANCE)
+    retractionTrait != null && retractionTrait.sendsUpdatesAsRetractions
+  }
+
+  /**
+    * Rule that assigns the default retraction information to [[DataStreamRel]] nodes.
+    * The default is to not publish updates as retraction messages and [[AccMode.Acc]].
+    */
+  class AssignDefaultRetractionRule extends RelOptRule(
+    operand(
+      classOf[DataStreamRel], none()),
+    "AssignDefaultRetractionRule") {
+
+    override def onMatch(call: RelOptRuleCall): Unit = {
+      val rel = call.rel(0).asInstanceOf[DataStreamRel]
+      val traits = rel.getTraitSet
+
+      val traitsWithUpdateAsRetrac = if (!traits.contains(UpdateAsRetractionTraitDef.INSTANCE)) {
+        traits.plus(UpdateAsRetractionTrait.DEFAULT)
+      } else {
+        traits
+      }
+      val traitsWithAccMode =
+        if (!traitsWithUpdateAsRetrac.contains(AccModeTraitDef.INSTANCE)) {
+          traitsWithUpdateAsRetrac.plus(AccModeTrait.DEFAULT)
+      } else {
+        traitsWithUpdateAsRetrac
+      }
+
+      if (traits != traitsWithAccMode) {
+        call.transformTo(rel.copy(traitsWithAccMode, rel.getInputs))
+      }
+    }
+  }
+
+  /**
+    * Rule that annotates all [[DataStreamRel]] nodes that need to sent out update and delete
+    * changes as retraction messages.
+    */
+  class SetUpdatesAsRetractionRule extends RelOptRule(
+    operand(
+      classOf[DataStreamRel], none()),
+    "SetUpdatesAsRetractionRule") {
+
+    /**
+      * Checks if a [[RelNode]] requires that update and delete changes are sent with retraction
+      * messages.
+      */
+    def needsUpdatesAsRetraction(node: RelNode): Boolean = {
+      node match {
+        case _ if sendsUpdatesAsRetraction(node) => true
+        case dsr: DataStreamRel => dsr.needsUpdatesAsRetraction
+      }
+    }
+
+    /**
+      * Annotates a [[RelNode]] to send out update and delete changes as retraction messages.
+      */
+    def setUpdatesAsRetraction(relNode: RelNode): RelNode = {
+      val traitSet = relNode.getTraitSet
+      relNode.copy(traitSet.plus(new UpdateAsRetractionTrait(true)), relNode.getInputs)
+    }
+
+    /**
+      * Annotates the children of a parent node with the information that they need to forward
+      * update and delete modifications as retraction messages.
+      *
+      * A child needs to produce retraction messages, if
+      *
+      * 1. its parent requires retraction messages by itself because it is a certain type
+      *    of operator, such as a [[DataStreamGroupAggregate]] or [[DataStreamOverAggregate]], or
+      * 2. its parent requires retraction because its own parent requires retraction
+      *    (transitive requirement).
+      *
+      */
+    override def onMatch(call: RelOptRuleCall): Unit = {
+      val parent = call.rel(0).asInstanceOf[DataStreamRel]
+
+      val children = getChildRelNodes(parent)
+      // check if children need to sent out retraction messages
+      val newChildren = for (c <- children) yield {
+        if (needsUpdatesAsRetraction(parent) && !sendsUpdatesAsRetraction(c)) {
+          setUpdatesAsRetraction(c)
+        } else {
+          c
+        }
+      }
+
+      // update parent if a child was updated
+      if (children != newChildren) {
+        call.transformTo(parent.copy(parent.getTraitSet, newChildren.asJava))
+      }
+    }
+  }
+
+  /**
+    * Sets the [[AccMode]] of [[DataStreamRel]] nodes.
+    */
+  class SetAccModeRule extends RelOptRule(
+    operand(
+      classOf[DataStreamRel], none()),
+    "SetAccModeRule") {
+
+    /**
+      * Checks if a [[RelNode]] produces update and delete changes.
+      */
+    def producesUpdates(relNode: RelNode): Boolean = {
+      relNode match {
+        case dsr: DataStreamRel => dsr.producesUpdates
+      }
+    }
+
+    /**
+      * Checks if a [[RelNode]] is in [[AccMode.AccRetract]] mode.
+      */
+    def isAccRetract(node: RelNode): Boolean = {
+      val accModeTrait = node.getTraitSet.getTrait(AccModeTraitDef.INSTANCE)
+      null != accModeTrait && accModeTrait.getAccMode == AccMode.AccRetract
+    }
+
+    /**
+      * Set [[AccMode.AccRetract]] to a [[RelNode]].
+      */
+    def setAccRetract(relNode: RelNode): RelNode = {
+      val traitSet = relNode.getTraitSet
+      relNode.copy(traitSet.plus(new AccModeTrait(AccMode.AccRetract)), relNode.getInputs)
+    }
+
+    /**
+      * Checks if a [[RelNode]] consumes retraction messages instead of forwarding them.
+      * The node might or might not produce new retraction messages.
+      * This is checked by [[producesRetractions()]].
+      */
+    def consumesRetractions(relNode: RelNode): Boolean = {
+      relNode match {
+        case dsr: DataStreamRel => dsr.consumesRetractions
+      }
+    }
+
+    /**
+      * Checks if a [[RelNode]] produces retraction messages.
+      */
+    def producesRetractions(node: RelNode): Boolean = {
+      sendsUpdatesAsRetraction(node) && producesUpdates(node)
+    }
+
+    /**
+      * Checks if a [[RelNode]] forwards retraction messages from its children.
+      */
+    def forwardsRetractions(parent: RelNode, children: Seq[RelNode]): Boolean = {
+      children.exists(c => isAccRetract(c)) && !consumesRetractions(parent)
+    }
+
+    /**
+      * Updates the [[AccMode]] of a [[RelNode]] and its children if necessary.
+      */
+    override def onMatch(call: RelOptRuleCall): Unit = {
+      val parent = call.rel(0).asInstanceOf[DataStreamRel]
+      val children = getChildRelNodes(parent)
+
+      // check if the AccMode of the parent needs to be updated
+      if (!isAccRetract(parent) &&
+          (producesRetractions(parent) || forwardsRetractions(parent, children))) {
+        call.transformTo(setAccRetract(parent))
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
index ed29f0f..0c337e0 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
@@ -22,6 +22,7 @@ import org.apache.calcite.rel.rules._
 import org.apache.calcite.sql.fun.{OracleSqlOperatorTable, SqlStdOperatorTable}
 import org.apache.calcite.tools.RuleSets
 import org.apache.flink.table.calcite.{CalciteConfig, CalciteConfigBuilder}
+import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules
 import org.junit.Assert._
 import org.junit.Test
 
@@ -51,11 +52,11 @@ class CalciteConfigBuilderTest {
   def testRules(): Unit = {
 
     val cc: CalciteConfig = new CalciteConfigBuilder()
-        .addNormRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
-        .replaceLogicalOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
-        .replacePhysicalOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
-        .replaceDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
-        .build()
+      .addNormRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .replaceLogicalOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .replacePhysicalOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .replaceDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
+      .build()
 
     assertFalse(cc.replacesNormRuleSet)
     assertTrue(cc.getNormRuleSet.isDefined)
@@ -232,62 +233,62 @@ class CalciteConfigBuilderTest {
   def testReplaceDecorationRules(): Unit = {
 
     val cc: CalciteConfig = new CalciteConfigBuilder()
-      .replaceDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .replaceDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
       .build()
 
     assertEquals(true, cc.replacesDecoRuleSet)
     assertTrue(cc.getDecoRuleSet.isDefined)
     val cSet = cc.getDecoRuleSet.get.iterator().asScala.toSet
     assertEquals(1, cSet.size)
-    assertTrue(cSet.contains(ReduceExpressionsRule.FILTER_INSTANCE))
+    assertTrue(cSet.contains(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
   }
 
   @Test
   def testReplaceDecorationAddRules(): Unit = {
 
     val cc: CalciteConfig = new CalciteConfigBuilder()
-      .replaceDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
-      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.PROJECT_INSTANCE))
+      .replaceDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
+      .addDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE))
       .build()
 
     assertEquals(true, cc.replacesDecoRuleSet)
     assertTrue(cc.getDecoRuleSet.isDefined)
     val cSet = cc.getDecoRuleSet.get.iterator().asScala.toSet
     assertEquals(2, cSet.size)
-    assertTrue(cSet.contains(ReduceExpressionsRule.FILTER_INSTANCE))
-    assertTrue(cSet.contains(ReduceExpressionsRule.PROJECT_INSTANCE))
+    assertTrue(cSet.contains(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
+    assertTrue(cSet.contains(DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE))
   }
 
   @Test
   def testAddDecorationRules(): Unit = {
 
     val cc: CalciteConfig = new CalciteConfigBuilder()
-      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .addDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
       .build()
 
     assertEquals(false, cc.replacesDecoRuleSet)
     assertTrue(cc.getDecoRuleSet.isDefined)
     val cSet = cc.getDecoRuleSet.get.iterator().asScala.toSet
     assertEquals(1, cSet.size)
-    assertTrue(cSet.contains(ReduceExpressionsRule.FILTER_INSTANCE))
+    assertTrue(cSet.contains(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
   }
 
   @Test
   def testAddAddDecorationRules(): Unit = {
 
     val cc: CalciteConfig = new CalciteConfigBuilder()
-      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
-      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.PROJECT_INSTANCE,
-                                      ReduceExpressionsRule.CALC_INSTANCE))
+      .addDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE))
+      .addDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE,
+                                      DataStreamRetractionRules.ACCMODE_INSTANCE))
       .build()
 
     assertEquals(false, cc.replacesDecoRuleSet)
     assertTrue(cc.getDecoRuleSet.isDefined)
     val cList = cc.getDecoRuleSet.get.iterator().asScala.toList
     assertEquals(3, cList.size)
-    assertEquals(cList.head, ReduceExpressionsRule.FILTER_INSTANCE)
-    assertEquals(cList(1), ReduceExpressionsRule.PROJECT_INSTANCE)
-    assertEquals(cList(2), ReduceExpressionsRule.CALC_INSTANCE)
+    assertEquals(cList.head, DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE)
+    assertEquals(cList(1), DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE)
+    assertEquals(cList(2), DataStreamRetractionRules.ACCMODE_INSTANCE)
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/flink/blob/dc54abc6/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
new file mode 100644
index 0000000..580029f
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/RetractionRulesTest.scala
@@ -0,0 +1,321 @@
+/*
+ * 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.table.plan.rules
+
+import org.apache.calcite.rel.RelNode
+import org.apache.flink.table.api.Table
+import org.apache.flink.table.plan.nodes.datastream._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Assert._
+import org.junit.{Ignore, Test}
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+
+
+class RetractionRulesTest extends TableTestBase {
+
+  def streamTestForRetractionUtil(): StreamTableTestForRetractionUtil = {
+    new StreamTableTestForRetractionUtil()
+  }
+
+  @Test
+  def testSelect(): Unit = {
+    val util = streamTestForRetractionUtil()
+    val table = util.addTable[(String, Int)]('word, 'number)
+
+    val resultTable = table.select('word, 'number)
+
+    val expected = s"DataStreamScan(false, Acc)"
+
+    util.verifyTableTrait(resultTable, expected)
+  }
+
+  // one level unbounded groupBy
+  @Test
+  def testGroupBy(): Unit = {
+    val util = streamTestForRetractionUtil()
+    val table = util.addTable[(String, Int)]('word, 'number)
+    val defaultStatus = "false, Acc"
+
+    val resultTable = table
+      .groupBy('word)
+      .select('number.count)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          "DataStreamScan(true, Acc)",
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifyTableTrait(resultTable, expected)
+  }
+
+  // two level unbounded groupBy
+  @Test
+  def testTwoGroupBy(): Unit = {
+    val util = streamTestForRetractionUtil()
+    val table = util.addTable[(String, Int)]('word, 'number)
+    val defaultStatus = "false, Acc"
+
+    val resultTable = table
+      .groupBy('word)
+      .select('word, 'number.count as 'count)
+      .groupBy('count)
+      .select('count, 'count.count as 'frequency)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            unaryNode(
+              "DataStreamGroupAggregate",
+              "DataStreamScan(true, Acc)",
+              "true, AccRetract"
+            ),
+            "true, AccRetract"
+          ),
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifyTableTrait(resultTable, expected)
+  }
+
+  // group window
+  @Test
+  def testGroupWindow(): Unit = {
+    val util = streamTestForRetractionUtil()
+    val table = util.addTable[(String, Int)]('word, 'number, 'rowtime.rowtime)
+    val defaultStatus = "false, Acc"
+
+    val resultTable = table
+      .window(Tumble over 50.milli on 'rowtime as 'w)
+      .groupBy('w, 'word)
+      .select('word, 'number.count as 'count)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupWindowAggregate",
+          "DataStreamScan(true, Acc)",
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifyTableTrait(resultTable, expected)
+  }
+
+  // group window after unbounded groupBy
+  @Test
+  @Ignore // cannot pass rowtime through non-windowed aggregation
+  def testGroupWindowAfterGroupBy(): Unit = {
+    val util = streamTestForRetractionUtil()
+    val table = util.addTable[(String, Int)]('word, 'number, 'rowtime.rowtime)
+    val defaultStatus = "false, Acc"
+
+    val resultTable = table
+      .groupBy('word)
+      .select('word, 'number.count as 'count)
+      .window(Tumble over 50.milli on 'rowtime as 'w)
+      .groupBy('w, 'count)
+      .select('count, 'count.count as 'frequency)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupWindowAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            unaryNode(
+              "DataStreamGroupAggregate",
+              "DataStreamScan(true, Acc)",
+              "true, AccRetract"
+            ),
+            "true, AccRetract"
+          ),
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifyTableTrait(resultTable, expected)
+  }
+
+  // over window
+  @Test
+  def testOverWindow(): Unit = {
+    val util = streamTestForRetractionUtil()
+    util.addTable[(String, Int)]("T1", 'word, 'number, 'proctime.proctime)
+    val defaultStatus = "false, Acc"
+
+    val sqlQuery =
+      "SELECT " +
+        "word, count(number) " +
+        "OVER (PARTITION BY word ORDER BY proctime " +
+        "ROWS BETWEEN UNBOUNDED preceding AND CURRENT ROW)" +
+        "FROM T1"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          "DataStreamScan(true, Acc)",
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifySqlTrait(sqlQuery, expected)
+  }
+
+
+  // over window after unbounded groupBy
+  @Test
+  @Ignore // cannot pass rowtime through non-windowed aggregation
+  def testOverWindowAfterGroupBy(): Unit = {
+    val util = streamTestForRetractionUtil()
+    util.addTable[(String, Int)]("T1", 'word, 'number, 'proctime.proctime)
+    val defaultStatus = "false, Acc"
+
+    val sqlQuery =
+      "SELECT " +
+        "_count, count(word) " +
+        "OVER (PARTITION BY _count ORDER BY proctime " +
+        "ROWS BETWEEN UNBOUNDED preceding AND CURRENT ROW)" +
+        "FROM " +
+        "(SELECT word, count(number) as _count FROM T1 GROUP BY word) "
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            unaryNode(
+              "DataStreamGroupAggregate",
+              "DataStreamScan(true, Acc)",
+              "true, AccRetract"
+            ),
+            "true, AccRetract"
+          ),
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifySqlTrait(sqlQuery, expected)
+  }
+
+  // test binaryNode
+  @Test
+  def testBinaryNode(): Unit = {
+    val util = streamTestForRetractionUtil()
+    val lTable = util.addTable[(String, Int)]('word, 'number)
+    val rTable = util.addTable[(String, Long)]('word_r, 'count_r)
+    val defaultStatus = "false, Acc"
+
+    val resultTable = lTable
+      .groupBy('word)
+      .select('word, 'number.count as 'count)
+      .unionAll(rTable)
+      .groupBy('count)
+      .select('count, 'count.count as 'frequency)
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamGroupAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            binaryNode(
+              "DataStreamUnion",
+              unaryNode(
+                "DataStreamCalc",
+                unaryNode(
+                  "DataStreamGroupAggregate",
+                  "DataStreamScan(true, Acc)",
+                  "true, AccRetract"
+                ),
+                "true, AccRetract"
+              ),
+              "DataStreamScan(true, Acc)",
+              "true, AccRetract"
+            ),
+            "true, AccRetract"
+          ),
+          s"$defaultStatus"
+        ),
+        s"$defaultStatus"
+      )
+
+    util.verifyTableTrait(resultTable, expected)
+  }
+}
+
+class StreamTableTestForRetractionUtil extends StreamTableTestUtil {
+
+  def verifySqlTrait(query: String, expected: String): Unit = {
+    verifyTableTrait(tEnv.sql(query), expected)
+  }
+
+  def verifyTableTrait(resultTable: Table, expected: String): Unit = {
+    val relNode = resultTable.getRelNode
+    val optimized = tEnv.optimize(relNode)
+    val actual = TraitUtil.toString(optimized)
+    assertEquals(
+      expected.split("\n").map(_.trim).mkString("\n"),
+      actual.split("\n").map(_.trim).mkString("\n"))
+  }
+}
+
+object TraitUtil {
+  def toString(rel: RelNode): String = {
+    val className = rel.getClass.getSimpleName
+    var childString: String = ""
+    var i = 0
+    while (i < rel.getInputs.size()) {
+      childString += TraitUtil.toString(rel.getInput(i))
+      i += 1
+    }
+
+    val retractString = rel.getTraitSet.getTrait(UpdateAsRetractionTraitDef.INSTANCE).toString
+    val accModetString = rel.getTraitSet.getTrait(AccModeTraitDef.INSTANCE).toString
+
+    s"""$className($retractString, $accModetString)
+       |$childString
+       |""".stripMargin.stripLineEnd
+  }
+}
+


[03/15] flink git commit: [FLINK-5884] [table] Integrate time indicators for Table API & SQL

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 2224752..8eb9d40 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -20,36 +20,34 @@ package org.apache.flink.table.plan.nodes.datastream
 import java.util.{List => JList}
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING
 import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.{AggregateCall, Window}
 import org.apache.calcite.rel.core.Window.Group
+import org.apache.calcite.rel.core.{AggregateCall, Window}
 import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING
-import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.java.functions.NullByteKeySelector
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.nodes.OverAggregate
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
 import org.apache.flink.table.runtime.aggregate._
 import org.apache.flink.types.Row
 
-import org.apache.flink.api.java.functions.NullByteKeySelector
-import org.apache.flink.table.codegen.CodeGenerator
-import org.apache.flink.table.functions.{ProcTimeType, RowTimeType}
-import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
-
 class DataStreamOverAggregate(
     logicWindow: Window,
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
     inputNode: RelNode,
-    rowRelDataType: RelDataType,
-    inputType: RelDataType)
+    schema: RowSchema,
+    inputSchema: RowSchema)
   extends SingleRel(cluster, traitSet, inputNode)
   with OverAggregate
   with DataStreamRel {
 
-  override def deriveRowType(): RelDataType = rowRelDataType
+  override def deriveRowType(): RelDataType = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, inputs: JList[RelNode]): RelNode = {
     new DataStreamOverAggregate(
@@ -57,8 +55,8 @@ class DataStreamOverAggregate(
       cluster,
       traitSet,
       inputs.get(0),
-      getRowType,
-      inputType)
+      schema,
+      inputSchema)
   }
 
   override def toString: String = {
@@ -72,14 +70,16 @@ class DataStreamOverAggregate(
     val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
 
     super.explainTerms(pw)
-      .itemIf("partitionBy", partitionToString(inputType, partitionKeys), partitionKeys.nonEmpty)
-      .item("orderBy",orderingToString(inputType, overWindow.orderKeys.getFieldCollations))
-      .itemIf("rows", windowRange(logicWindow, overWindow, getInput), overWindow.isRows)
-      .itemIf("range", windowRange(logicWindow, overWindow, getInput), !overWindow.isRows)
+      .itemIf("partitionBy",
+        partitionToString(schema.logicalType, partitionKeys), partitionKeys.nonEmpty)
+      .item("orderBy",
+        orderingToString(schema.logicalType, overWindow.orderKeys.getFieldCollations))
+      .itemIf("rows", windowRange(logicWindow, overWindow, inputNode), overWindow.isRows)
+      .itemIf("range", windowRange(logicWindow, overWindow, inputNode), !overWindow.isRows)
       .item(
         "select", aggregationToString(
-          inputType,
-          getRowType,
+          inputSchema.logicalType,
+          schema.logicalType,
           namedAggregates))
   }
 
@@ -111,13 +111,13 @@ class DataStreamOverAggregate(
       false,
       inputDS.getType)
 
-    val timeType = inputType
+    val timeType = schema.logicalType
       .getFieldList
       .get(orderKey.getFieldIndex)
-      .getValue
+      .getType
 
     timeType match {
-      case _: ProcTimeType =>
+      case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
         // proc-time OVER window
         if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
           // unbounded OVER window
@@ -140,7 +140,8 @@ class DataStreamOverAggregate(
           throw new TableException(
             "OVER RANGE FOLLOWING windows are not supported yet.")
         }
-      case _: RowTimeType =>
+
+      case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
         // row-time OVER window
         if (overWindow.lowerBound.isPreceding &&
           overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
@@ -158,17 +159,16 @@ class DataStreamOverAggregate(
             inputDS,
             isRowTimeType = true,
             isRowsClause = overWindow.isRows
-            )
+          )
         } else {
           throw new TableException(
             "OVER RANGE FOLLOWING windows are not supported yet.")
         }
+
       case _ =>
         throw new TableException(
-          "Unsupported time type {$timeType}. " +
-            "OVER windows do only support RowTimeType and ProcTimeType.")
+          s"OVER windows can only be applied on time attributes.")
     }
-
   }
 
   def createUnboundedAndCurrentRowOverWindow(
@@ -178,16 +178,20 @@ class DataStreamOverAggregate(
     isRowsClause: Boolean): DataStream[Row] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
-    val partitionKeys: Array[Int] = overWindow.keys.toArray
-    val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
-
-    // get the output types
-    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+    val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex)
+    val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates.map {
+      namedAggregate =>
+        new CalcitePair[AggregateCall, String](
+          schema.mapAggregateCall(namedAggregate.left),
+          namedAggregate.right)
+    }
 
     val processFunction = AggregateUtil.createUnboundedOverProcessFunction(
       generator,
       namedAggregates,
-      inputType,
+      inputSchema.physicalType,
+      inputSchema.physicalTypeInfo,
+      inputSchema.physicalFieldTypeInfo,
       isRowTimeType,
       partitionKeys.nonEmpty,
       isRowsClause)
@@ -198,7 +202,7 @@ class DataStreamOverAggregate(
         inputDS
           .keyBy(partitionKeys: _*)
           .process(processFunction)
-          .returns(rowTypeInfo)
+          .returns(schema.physicalTypeInfo)
           .name(aggOpName)
           .asInstanceOf[DataStream[Row]]
       }
@@ -207,13 +211,13 @@ class DataStreamOverAggregate(
         if (isRowTimeType) {
           inputDS.keyBy(new NullByteKeySelector[Row])
             .process(processFunction).setParallelism(1).setMaxParallelism(1)
-            .returns(rowTypeInfo)
+            .returns(schema.physicalTypeInfo)
             .name(aggOpName)
             .asInstanceOf[DataStream[Row]]
         } else {
           inputDS
             .process(processFunction).setParallelism(1).setMaxParallelism(1)
-            .returns(rowTypeInfo)
+            .returns(schema.physicalTypeInfo)
             .name(aggOpName)
             .asInstanceOf[DataStream[Row]]
         }
@@ -228,19 +232,26 @@ class DataStreamOverAggregate(
     isRowsClause: Boolean): DataStream[Row] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
-    val partitionKeys: Array[Int] = overWindow.keys.toArray
-    val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
+    val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex)
+    val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates.map {
+      namedAggregate =>
+        new CalcitePair[AggregateCall, String](
+          schema.mapAggregateCall(namedAggregate.left),
+          namedAggregate.right)
+    }
 
     val precedingOffset =
-      getLowerBoundary(logicWindow, overWindow, getInput()) + (if (isRowsClause) 1 else 0)
-
-    // get the output types
-    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+      getLowerBoundary(
+        logicWindow,
+        overWindow,
+        input) + (if (isRowsClause) 1 else 0)
 
     val processFunction = AggregateUtil.createBoundedOverProcessFunction(
       generator,
       namedAggregates,
-      inputType,
+      inputSchema.physicalType,
+      inputSchema.physicalTypeInfo,
+      inputSchema.physicalFieldTypeInfo,
       precedingOffset,
       isRowsClause,
       isRowTimeType
@@ -251,7 +262,7 @@ class DataStreamOverAggregate(
         inputDS
           .keyBy(partitionKeys: _*)
           .process(processFunction)
-          .returns(rowTypeInfo)
+          .returns(schema.physicalTypeInfo)
           .name(aggOpName)
           .asInstanceOf[DataStream[Row]]
       }
@@ -260,7 +271,7 @@ class DataStreamOverAggregate(
         inputDS
           .keyBy(new NullByteKeySelector[Row])
           .process(processFunction).setParallelism(1).setMaxParallelism(1)
-          .returns(rowTypeInfo)
+          .returns(schema.physicalTypeInfo)
           .name(aggOpName)
           .asInstanceOf[DataStream[Row]]
       }
@@ -282,17 +293,18 @@ class DataStreamOverAggregate(
 
     s"over: (${
       if (!partitionKeys.isEmpty) {
-        s"PARTITION BY: ${partitionToString(inputType, partitionKeys)}, "
+        s"PARTITION BY: ${partitionToString(inputSchema.logicalType, partitionKeys)}, "
       } else {
         ""
       }
-    }ORDER BY: ${orderingToString(inputType, overWindow.orderKeys.getFieldCollations)}, " +
+    }ORDER BY: ${orderingToString(inputSchema.logicalType,
+        overWindow.orderKeys.getFieldCollations)}, " +
       s"${if (overWindow.isRows) "ROWS" else "RANGE"}" +
-      s"${windowRange(logicWindow, overWindow, getInput)}, " +
+      s"${windowRange(logicWindow, overWindow, inputNode.asInstanceOf[DataStreamRel])}, " +
       s"select: (${
         aggregationToString(
-          inputType,
-          getRowType,
+          inputSchema.logicalType,
+          schema.logicalType,
           namedAggregates)
       }))"
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
index ae172a5..03938f3 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
@@ -34,4 +34,3 @@ trait DataStreamRel extends FlinkRelNode {
   def translateToPlan(tableEnv: StreamTableEnvironment) : DataStream[Row]
 
 }
-

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
index c187ae8..05f60ba 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
@@ -24,7 +24,7 @@ import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.TableScan
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
-import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema}
 import org.apache.flink.types.Row
 
 /**
@@ -36,27 +36,27 @@ class DataStreamScan(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
     table: RelOptTable,
-    rowRelDataType: RelDataType)
+    schema: RowSchema)
   extends TableScan(cluster, traitSet, table)
   with StreamScan {
 
   val dataStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]])
 
-  override def deriveRowType(): RelDataType = rowRelDataType
+  override def deriveRowType(): RelDataType = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamScan(
       cluster,
       traitSet,
       getTable,
-      getRowType
+      schema
     )
   }
 
   override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
     val config = tableEnv.getConfig
     val inputDataStream: DataStream[Any] = dataStreamTable.dataStream
-    convertToInternalRow(inputDataStream, dataStreamTable, config)
+    convertToInternalRow(schema, inputDataStream, dataStreamTable, config)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
index f340ac7..47b4946 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
@@ -19,14 +19,12 @@
 package org.apache.flink.table.plan.nodes.datastream
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.types.Row
 
-import scala.collection.JavaConverters._
-
 /**
   * Flink RelNode which matches along with Union.
   *
@@ -36,11 +34,11 @@ class DataStreamUnion(
     traitSet: RelTraitSet,
     leftNode: RelNode,
     rightNode: RelNode,
-    rowRelDataType: RelDataType)
+    schema: RowSchema)
   extends BiRel(cluster, traitSet, leftNode, rightNode)
   with DataStreamRel {
 
-  override def deriveRowType() = rowRelDataType
+  override def deriveRowType() = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamUnion(
@@ -48,7 +46,7 @@ class DataStreamUnion(
       traitSet,
       inputs.get(0),
       inputs.get(1),
-      getRowType
+      schema
     )
   }
 
@@ -57,7 +55,7 @@ class DataStreamUnion(
   }
 
   override def toString = {
-    s"Union All(union: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
+    s"Union All(union: (${schema.logicalFieldNames.mkString(", ")}))"
   }
 
   override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
@@ -68,6 +66,6 @@ class DataStreamUnion(
   }
 
   private def unionSelectionToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
+    schema.logicalFieldNames.mkString(", ")
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
index 0ab4a48..c964e03 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
@@ -21,13 +21,12 @@ package org.apache.flink.table.plan.nodes.datastream
 import com.google.common.collect.ImmutableList
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.Values
 import org.apache.calcite.rex.RexLiteral
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.StreamTableEnvironment
-import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.io.ValuesInputFormat
 import org.apache.flink.types.Row
 
@@ -39,19 +38,19 @@ import scala.collection.JavaConverters._
 class DataStreamValues(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
-    rowRelDataType: RelDataType,
+    schema: RowSchema,
     tuples: ImmutableList[ImmutableList[RexLiteral]],
     ruleDescription: String)
-  extends Values(cluster, rowRelDataType, tuples, traitSet)
+  extends Values(cluster, schema.logicalType, tuples, traitSet)
   with DataStreamRel {
 
-  override def deriveRowType() = rowRelDataType
+  override def deriveRowType() = schema.logicalType
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new DataStreamValues(
       cluster,
       traitSet,
-      getRowType,
+      schema,
       getTuples,
       ruleDescription
     )
@@ -61,15 +60,13 @@ class DataStreamValues(
 
     val config = tableEnv.getConfig
 
-    val returnType = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
-
     val generator = new CodeGenerator(config)
 
     // generate code for every record
     val generatedRecords = getTuples.asScala.map { r =>
       generator.generateResultExpression(
-        returnType,
-        getRowType.getFieldNames.asScala,
+        schema.physicalTypeInfo,
+        schema.physicalFieldNames,
         r.asScala)
     }
 
@@ -77,14 +74,14 @@ class DataStreamValues(
     val generatedFunction = generator.generateValuesInputFormat(
       ruleDescription,
       generatedRecords.map(_.code),
-      returnType)
+      schema.physicalTypeInfo)
 
     val inputFormat = new ValuesInputFormat[Row](
       generatedFunction.name,
       generatedFunction.code,
       generatedFunction.returnType)
 
-    tableEnv.execEnv.createInput(inputFormat, returnType)
+    tableEnv.execEnv.createInput(inputFormat, schema.physicalTypeInfo)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
index 6d08302..dd82819 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
@@ -18,42 +18,46 @@
 
 package org.apache.flink.table.plan.nodes.datastream
 
+import org.apache.flink.api.common.functions.MapFunction
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.TableConfig
-import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.CommonScan
-import org.apache.flink.table.plan.schema.FlinkTable
+import org.apache.flink.table.plan.schema.{FlinkTable, RowSchema}
+import org.apache.flink.table.runtime.MapRunner
 import org.apache.flink.types.Row
 
-import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
 
 trait StreamScan extends CommonScan with DataStreamRel {
 
   protected def convertToInternalRow(
+      schema: RowSchema,
       input: DataStream[Any],
       flinkTable: FlinkTable[_],
       config: TableConfig)
     : DataStream[Row] = {
 
-    val inputType = input.getType
-
-    val internalType = FlinkTypeFactory.toInternalRowTypeInfo(getRowType)
-
     // conversion
-    if (needsConversion(inputType, internalType)) {
+    if (needsConversion(input.getType, schema.physicalTypeInfo)) {
 
-      val mapFunc = getConversionMapper(
+      val function = generatedConversionFunction(
         config,
-        inputType,
-        internalType,
+        classOf[MapFunction[Any, Row]],
+        input.getType,
+        schema.physicalTypeInfo,
         "DataStreamSourceConversion",
-        getRowType.getFieldNames,
+        schema.physicalFieldNames,
         Some(flinkTable.fieldIndexes))
 
-      val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+      val runner = new MapRunner[Any, Row](
+        function.name,
+        function.code,
+        function.returnType)
+
+      val opName = s"from: (${schema.logicalFieldNames.mkString(", ")})"
 
-      input.map(mapFunc).name(opName)
+      // TODO we need a ProcessFunction here
+      input.map(runner).name(opName)
     }
     // no conversion necessary, forward
     else {

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
index 0a466a3..5dc3da8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
@@ -22,10 +22,11 @@ import org.apache.calcite.plan._
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment}
+import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.PhysicalTableSourceScan
-import org.apache.flink.table.plan.schema.TableSourceTable
-import org.apache.flink.table.sources.{StreamTableSource, TableSource}
+import org.apache.flink.table.plan.schema.{RowSchema, TableSourceTable}
+import org.apache.flink.table.sources.{DefinedTimeAttributes, StreamTableSource, TableSource}
 import org.apache.flink.types.Row
 
 /** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */
@@ -37,7 +38,50 @@ class StreamTableSourceScan(
   extends PhysicalTableSourceScan(cluster, traitSet, table, tableSource)
   with StreamScan {
 
-  override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+  override def deriveRowType() = {
+    val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+
+    def removeIndex[T](idx: Int, l: List[T]): List[T] = {
+      if (l.size < idx) {
+        l
+      } else {
+        l.take(idx) ++ l.drop(idx + 1)
+      }
+    }
+
+    var fieldNames = TableEnvironment.getFieldNames(tableSource).toList
+    var fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList
+
+    val rowtime = tableSource match {
+      case timeSource: DefinedTimeAttributes if timeSource.getRowtimeAttribute != null =>
+        val rowtimeAttribute = timeSource.getRowtimeAttribute
+        // remove physical field if it is overwritten by time attribute
+        fieldNames = removeIndex(rowtimeAttribute.f0, fieldNames)
+        fieldTypes = removeIndex(rowtimeAttribute.f0, fieldTypes)
+        Some((rowtimeAttribute.f0, rowtimeAttribute.f1))
+      case _ =>
+        None
+    }
+
+    val proctime = tableSource match {
+      case timeSource: DefinedTimeAttributes if timeSource.getProctimeAttribute != null =>
+        val proctimeAttribute = timeSource.getProctimeAttribute
+        // remove physical field if it is overwritten by time attribute
+        fieldNames = removeIndex(proctimeAttribute.f0, fieldNames)
+        fieldTypes = removeIndex(proctimeAttribute.f0, fieldTypes)
+        Some((proctimeAttribute.f0, proctimeAttribute.f1))
+      case _ =>
+        None
+    }
+
+    flinkTypeFactory.buildLogicalRowType(
+      fieldNames,
+      fieldTypes,
+      rowtime,
+      proctime)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
     val rowCnt = metadata.getRowCount(this)
     planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType))
   }
@@ -67,6 +111,10 @@ class StreamTableSourceScan(
   override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[Row] = {
     val config = tableEnv.getConfig
     val inputDataStream = tableSource.getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]]
-    convertToInternalRow(inputDataStream, new TableSourceTable(tableSource), config)
+    convertToInternalRow(
+      new RowSchema(getRowType),
+      inputDataStream,
+      new TableSourceTable(tableSource),
+      config)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalOverWindow.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalOverWindow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalOverWindow.scala
index b1f991e..11b227f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalOverWindow.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalOverWindow.scala
@@ -45,7 +45,7 @@ class FlinkLogicalOverWindow(
       traitSet,
       inputs.get(0),
       windowConstants,
-      rowType,
+      getRowType,
       windowGroups)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
index eacbafa..53e7b31 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
@@ -47,9 +47,11 @@ class FlinkLogicalTableSourceScan(
 
   override def deriveRowType(): RelDataType = {
     val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    flinkTypeFactory.buildRowDataType(
+    flinkTypeFactory.buildLogicalRowType(
       TableEnvironment.getFieldNames(tableSource),
-      TableEnvironment.getFieldTypes(tableSource.getReturnType))
+      TableEnvironment.getFieldTypes(tableSource.getReturnType),
+      None,
+      None)
   }
 
   override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowStartEndPropertiesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowStartEndPropertiesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowStartEndPropertiesRule.scala
index 4da2da9..7577deb 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowStartEndPropertiesRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowStartEndPropertiesRule.scala
@@ -53,8 +53,8 @@ class WindowStartEndPropertiesRule
     transformed.push(LogicalWindowAggregate.create(
       agg.getWindow,
       Seq(
-        NamedWindowProperty("w$start", WindowStart(agg.getWindow.alias)),
-        NamedWindowProperty("w$end", WindowEnd(agg.getWindow.alias))
+        NamedWindowProperty("w$start", WindowStart(agg.getWindow.aliasAttribute)),
+        NamedWindowProperty("w$end", WindowEnd(agg.getWindow.aliasAttribute))
       ), agg)
     )
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
index f011b66..fc65403 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
@@ -25,6 +25,7 @@ import org.apache.flink.table.api.TableException
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamAggregate
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalWindowAggregate
+import org.apache.flink.table.plan.schema.RowSchema
 
 import scala.collection.JavaConversions._
 
@@ -65,8 +66,8 @@ class DataStreamAggregateRule
       traitSet,
       convInput,
       agg.getNamedAggCalls,
-      rel.getRowType,
-      agg.getInput.getRowType,
+      new RowSchema(rel.getRowType),
+      new RowSchema(agg.getInput.getRowType),
       agg.getGroupSet.toArray)
     }
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
index 1777264..0a1a31a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamCalc
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc
+import org.apache.flink.table.plan.schema.RowSchema
 
 class DataStreamCalcRule
   extends ConverterRule(
@@ -42,7 +43,8 @@ class DataStreamCalcRule
       rel.getCluster,
       traitSet,
       convInput,
-      rel.getRowType,
+      new RowSchema(convInput.getRowType),
+      new RowSchema(rel.getRowType),
       calc.getProgram,
       description)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
index ae39d40..cd0663e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
@@ -25,6 +25,7 @@ import org.apache.calcite.rex.RexNode
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamCorrelate
 import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalCorrelate, FlinkLogicalTableFunctionScan}
+import org.apache.flink.table.plan.schema.RowSchema
 
 class DataStreamCorrelateRule
   extends ConverterRule(
@@ -68,11 +69,12 @@ class DataStreamCorrelateRule
           new DataStreamCorrelate(
             rel.getCluster,
             traitSet,
+            new RowSchema(convInput.getRowType),
             convInput,
             scan,
             condition,
-            rel.getRowType,
-            join.getRowType,
+            new RowSchema(rel.getRowType),
+            new RowSchema(join.getRowType),
             join.getJoinType,
             description)
       }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
index 175a202..28efcf5 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
@@ -18,15 +18,15 @@
 
 package org.apache.flink.table.plan.rules.datastream
 
-import java.math.BigDecimal
+import java.math.{BigDecimal => JBigDecimal}
 
 import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rex.{RexBuilder, RexCall, RexLiteral, RexNode}
+import org.apache.calcite.rex._
 import org.apache.calcite.sql.fun.SqlStdOperatorTable
 import org.apache.flink.table.api.{TableException, Window}
 import org.apache.flink.table.api.scala.{Session, Slide, Tumble}
-import org.apache.flink.table.expressions.Literal
-import org.apache.flink.table.functions.TimeModeTypes
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions.{Literal, UnresolvedFieldReference}
 import org.apache.flink.table.plan.rules.common.LogicalWindowAggregateRule
 import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
 
@@ -49,16 +49,12 @@ class DataStreamLogicalWindowAggregateRule
 
     val timeType = windowExpression.operands.get(0).getType
     timeType match {
-      case TimeModeTypes.ROWTIME =>
-        rexBuilder.makeAbstractCast(
-          TimeModeTypes.ROWTIME,
-          rexBuilder.makeLiteral(0L, TimeModeTypes.ROWTIME, true))
-      case TimeModeTypes.PROCTIME =>
-        rexBuilder.makeAbstractCast(
-          TimeModeTypes.PROCTIME,
-          rexBuilder.makeLiteral(0L, TimeModeTypes.PROCTIME, true))
+
+      case _ if FlinkTypeFactory.isTimeIndicatorType(timeType) =>
+        rexBuilder.makeLiteral(0L, timeType, true)
+
       case _ =>
-        throw TableException(s"""Unexpected time type $timeType encountered""")
+        throw TableException(s"""Time attribute expected but $timeType encountered.""")
     }
   }
 
@@ -68,41 +64,41 @@ class DataStreamLogicalWindowAggregateRule
 
     def getOperandAsLong(call: RexCall, idx: Int): Long =
       call.getOperands.get(idx) match {
-        case v : RexLiteral => v.getValue.asInstanceOf[BigDecimal].longValue()
-        case _ => throw new TableException("Only constant window descriptors are supported")
+        case v: RexLiteral => v.getValue.asInstanceOf[JBigDecimal].longValue()
+        case _ => throw new TableException("Only constant window descriptors are supported.")
+      }
+
+    def getOperandAsTimeIndicator(call: RexCall, idx: Int): String =
+      call.getOperands.get(idx) match {
+        case v: RexInputRef if FlinkTypeFactory.isTimeIndicatorType(v.getType) =>
+          rowType.getFieldList.get(v.getIndex).getName
+        case _ =>
+          throw new TableException("Window can only be defined over a time attribute column.")
       }
 
     windowExpr.getOperator match {
       case SqlStdOperatorTable.TUMBLE =>
+        val time = getOperandAsTimeIndicator(windowExpr, 0)
         val interval = getOperandAsLong(windowExpr, 1)
         val w = Tumble.over(Literal(interval, TimeIntervalTypeInfo.INTERVAL_MILLIS))
 
-        val window = windowExpr.getType match {
-          case TimeModeTypes.PROCTIME => w
-          case TimeModeTypes.ROWTIME => w.on("rowtime")
-        }
-        window.as("w$")
+        w.on(UnresolvedFieldReference(time)).as("w$")
 
       case SqlStdOperatorTable.HOP =>
+        val time = getOperandAsTimeIndicator(windowExpr, 0)
         val (slide, size) = (getOperandAsLong(windowExpr, 1), getOperandAsLong(windowExpr, 2))
         val w = Slide
           .over(Literal(size, TimeIntervalTypeInfo.INTERVAL_MILLIS))
           .every(Literal(slide, TimeIntervalTypeInfo.INTERVAL_MILLIS))
 
-        val window = windowExpr.getType match {
-          case TimeModeTypes.PROCTIME => w
-          case TimeModeTypes.ROWTIME => w.on("rowtime")
-        }
-        window.as("w$")
+        w.on(UnresolvedFieldReference(time)).as("w$")
+
       case SqlStdOperatorTable.SESSION =>
+        val time = getOperandAsTimeIndicator(windowExpr, 0)
         val gap = getOperandAsLong(windowExpr, 1)
         val w = Session.withGap(Literal(gap, TimeIntervalTypeInfo.INTERVAL_MILLIS))
 
-        val window = windowExpr.getType match {
-          case TimeModeTypes.PROCTIME => w
-          case TimeModeTypes.ROWTIME => w.on("rowtime")
-        }
-        window.as("w$")
+        w.on(UnresolvedFieldReference(time)).as("w$")
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamOverAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamOverAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamOverAggregateRule.scala
index 8e96970..b3d7603 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamOverAggregateRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamOverAggregateRule.scala
@@ -25,6 +25,7 @@ import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamOverAggregate
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalOverWindow
+import org.apache.flink.table.plan.schema.RowSchema
 
 class DataStreamOverAggregateRule
   extends ConverterRule(
@@ -46,8 +47,8 @@ class DataStreamOverAggregateRule
       rel.getCluster,
       traitSet,
       convertInput,
-      rel.getRowType,
-      inputRowType)
+      new RowSchema(rel.getRowType),
+      new RowSchema(inputRowType))
   }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
index 5bf60a7..d8dda80 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
@@ -23,7 +23,7 @@ import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamScan
-import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema}
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalNativeTableScan
 
 class DataStreamScanRule
@@ -53,7 +53,7 @@ class DataStreamScanRule
       rel.getCluster,
       traitSet,
       scan.getTable,
-      rel.getRowType
+      new RowSchema(rel.getRowType)
     )
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
index 4241f53..8402f6d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamUnion
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalUnion
+import org.apache.flink.table.plan.schema.RowSchema
 
 class DataStreamUnionRule
   extends ConverterRule(
@@ -44,7 +45,7 @@ class DataStreamUnionRule
       traitSet,
       convLeft,
       convRight,
-      rel.getRowType)
+      new RowSchema(rel.getRowType))
   }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
index fbad21f..a1453a8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.datastream.DataStreamValues
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalValues
+import org.apache.flink.table.plan.schema.RowSchema
 
 class DataStreamValuesRule
   extends ConverterRule(
@@ -40,7 +41,7 @@ class DataStreamValuesRule
     new DataStreamValues(
       rel.getCluster,
       traitSet,
-      rel.getRowType,
+      new RowSchema(rel.getRowType),
       values.getTuples,
       description)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
index 6ce6570..70054b4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
@@ -18,13 +18,27 @@
 
 package org.apache.flink.table.plan.schema
 
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
 import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.stats.FlinkStatistic
 
 class DataStreamTable[T](
     val dataStream: DataStream[T],
     override val fieldIndexes: Array[Int],
     override val fieldNames: Array[String],
+    val rowtime: Option[(Int, String)],
+    val proctime: Option[(Int, String)],
     override val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN)
   extends FlinkTable[T](dataStream.getType, fieldIndexes, fieldNames, statistic) {
+
+  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = {
+    val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
+
+    flinkTypeFactory.buildLogicalRowType(
+      fieldNames,
+      fieldTypes,
+      rowtime,
+      proctime)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
index ea77061..752b00e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
@@ -48,10 +48,11 @@ abstract class FlinkTable[T](
   val fieldTypes: Array[TypeInformation[_]] =
     typeInfo match {
       case cType: CompositeType[_] =>
-        if (fieldNames.length != cType.getArity) {
+        // it is ok to leave out fields
+        if (fieldNames.length > cType.getArity) {
           throw new TableException(
           s"Arity of type (" + cType.getFieldNames.deep + ") " +
-            "not equal to number of field names " + fieldNames.deep + ".")
+            "must not be greater than number of field names " + fieldNames.deep + ".")
         }
         fieldIndexes.map(cType.getTypeAt(_).asInstanceOf[TypeInformation[_]])
       case aType: AtomicType[_] =>
@@ -64,7 +65,7 @@ abstract class FlinkTable[T](
 
   override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = {
     val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
-    flinkTypeFactory.buildRowDataType(fieldNames, fieldTypes)
+    flinkTypeFactory.buildLogicalRowType(fieldNames, fieldTypes, None, None)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
new file mode 100644
index 0000000..b42be82
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
@@ -0,0 +1,152 @@
+/*
+ * 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.table.plan.schema
+
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, RelRecordType}
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rex.{RexInputRef, RexNode, RexShuttle}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * Schema that describes both a logical and physical row.
+  */
+class RowSchema(private val logicalRowType: RelDataType) {
+
+  private lazy val physicalRowFields: Seq[RelDataTypeField] = logicalRowType.getFieldList filter {
+    field => !FlinkTypeFactory.isTimeIndicatorType(field.getType)
+  }
+
+  private lazy val physicalRowType: RelDataType = new RelRecordType(physicalRowFields)
+
+  private lazy val physicalRowFieldTypes: Seq[TypeInformation[_]] = physicalRowFields map { f =>
+    FlinkTypeFactory.toTypeInfo(f.getType)
+  }
+
+  private lazy val physicalRowFieldNames: Seq[String] = physicalRowFields.map(_.getName)
+
+  private lazy val physicalRowTypeInfo: TypeInformation[Row] = new RowTypeInfo(
+    physicalRowFieldTypes.toArray, physicalRowFieldNames.toArray)
+
+  private lazy val indexMapping: Array[Int] = generateIndexMapping
+
+  private lazy val inputRefUpdater = new RexInputRefUpdater()
+
+  private def generateIndexMapping: Array[Int] = {
+    val mapping = new Array[Int](logicalRowType.getFieldCount)
+    var countTimeIndicators = 0
+    var i = 0
+    while (i < logicalRowType.getFieldCount) {
+      val t = logicalRowType.getFieldList.get(i).getType
+      if (FlinkTypeFactory.isTimeIndicatorType(t)) {
+        countTimeIndicators += 1
+        // no mapping
+        mapping(i) = -1
+      } else {
+        mapping(i) = i - countTimeIndicators
+      }
+      i += 1
+    }
+    mapping
+  }
+
+  private class RexInputRefUpdater extends RexShuttle {
+
+    override def visitInputRef(inputRef: RexInputRef): RexNode = {
+      new RexInputRef(mapIndex(inputRef.getIndex), inputRef.getType)
+    }
+  }
+
+  /**
+    * Returns the arity of the logical record.
+    */
+  def logicalArity: Int = logicalRowType.getFieldCount
+
+  /**
+    * Returns the arity of the physical record.
+    */
+  def physicalArity: Int = physicalTypeInfo.getArity
+
+  /**
+    * Returns a logical [[RelDataType]] including logical fields (i.e. time indicators).
+    */
+  def logicalType: RelDataType = logicalRowType
+
+  /**
+    * Returns a physical [[RelDataType]] with no logical fields (i.e. time indicators).
+    */
+  def physicalType: RelDataType = physicalRowType
+
+  /**
+    * Returns a physical [[TypeInformation]] of row with no logical fields (i.e. time indicators).
+    */
+  def physicalTypeInfo: TypeInformation[Row] = physicalRowTypeInfo
+
+  /**
+    * Returns [[TypeInformation]] of the row's fields with no logical fields (i.e. time indicators).
+    */
+  def physicalFieldTypeInfo: Seq[TypeInformation[_]] = physicalRowFieldTypes
+
+  /**
+    * Returns the logical fields names including logical fields (i.e. time indicators).
+    */
+  def logicalFieldNames: Seq[String] = logicalRowType.getFieldNames
+
+  /**
+    * Returns the physical fields names with no logical fields (i.e. time indicators).
+    */
+  def physicalFieldNames: Seq[String] = physicalRowFieldNames
+
+  /**
+    * Converts logical indices to physical indices based on this schema.
+    */
+  def mapIndex(logicalIndex: Int): Int = {
+    val mappedIndex = indexMapping(logicalIndex)
+    if (mappedIndex < 0) {
+      throw new TableException("Invalid access to a logical field.")
+    } else {
+      mappedIndex
+    }
+  }
+
+  /**
+    * Converts logical indices of a aggregate call to physical ones.
+    */
+  def mapAggregateCall(logicalAggCall: AggregateCall): AggregateCall = {
+    logicalAggCall.copy(
+      logicalAggCall.getArgList.map(mapIndex(_).asInstanceOf[Integer]),
+      if (logicalAggCall.filterArg < 0) {
+        logicalAggCall.filterArg
+      } else {
+        mapIndex(logicalAggCall.filterArg)
+      }
+    )
+  }
+
+  /**
+    * Converts logical field references of a [[RexNode]] to physical ones.
+    */
+  def mapRexNode(logicalRexNode: RexNode): RexNode = logicalRexNode.accept(inputRefUpdater)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TimeIndicatorRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TimeIndicatorRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TimeIndicatorRelDataType.scala
new file mode 100644
index 0000000..5e27061
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TimeIndicatorRelDataType.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.table.plan.schema
+
+import org.apache.calcite.rel.`type`.RelDataTypeSystem
+import org.apache.calcite.sql.`type`.BasicSqlType
+
+/**
+  * Creates a time indicator type for event-time or processing-time, but with similar properties
+  * as a basic SQL type.
+  */
+class TimeIndicatorRelDataType(
+    typeSystem: RelDataTypeSystem,
+    originalType: BasicSqlType,
+    val isEventTime: Boolean)
+  extends BasicSqlType(
+    typeSystem,
+    originalType.getSqlTypeName,
+    originalType.getPrecision) {
+
+  override def equals(other: Any): Boolean = other match {
+    case that: TimeIndicatorRelDataType =>
+      super.equals(that) &&
+        isEventTime == that.isEventTime
+    case that: BasicSqlType =>
+      super.equals(that)
+    case _ => false
+  }
+
+  override def hashCode(): Int = {
+    super.hashCode() + 42 // we change the hash code to differentiate from regular timestamps
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
index 51e2fc5..32562c7 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
@@ -35,7 +35,7 @@ class MapRunner[IN, OUT](
 
   val LOG = LoggerFactory.getLogger(this.getClass)
 
-  private var function: MapFunction[IN, OUT] = null
+  private var function: MapFunction[IN, OUT] = _
 
   override def open(parameters: Configuration): Unit = {
     LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index e38207d..07992cd 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -26,17 +26,18 @@ import org.apache.calcite.sql.`type`.SqlTypeName
 import org.apache.calcite.sql.`type`.SqlTypeName._
 import org.apache.calcite.sql.fun._
 import org.apache.calcite.sql.{SqlAggFunction, SqlKind}
-import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.api.common.functions.{MapFunction, RichGroupReduceFunction, AggregateFunction => DataStreamAggFunction, _}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.common.functions.{AggregateFunction => DataStreamAggFunction, _}
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction}
 import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.expressions.ExpressionUtils.isTimeIntervalLiteral
 import org.apache.flink.table.expressions._
 import org.apache.flink.table.functions.aggfunctions._
 import org.apache.flink.table.functions.utils.AggSqlFunction
@@ -61,26 +62,31 @@ object AggregateUtil {
     * window to evaluate final aggregate value.
     *
     * @param generator       code generator instance
-    * @param namedAggregates List of calls to aggregate functions and their output field names
-    * @param inputType Input row type
+    * @param namedAggregates Physical calls to aggregate functions and their output field names
+    * @param inputType Physical type of the row.
+    * @param inputTypeInfo Physical type information of the row.
+    * @param inputFieldTypeInfo Physical type information of the row's fields.
     * @param isRowTimeType It is a tag that indicates whether the time type is rowTimeType
     * @param isPartitioned It is a tag that indicate whether the input is partitioned
     * @param isRowsClause It is a tag that indicates whether the OVER clause is ROWS clause
     */
   private[flink] def createUnboundedOverProcessFunction(
-    generator: CodeGenerator,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    isRowTimeType: Boolean,
-    isPartitioned: Boolean,
-    isRowsClause: Boolean): ProcessFunction[Row, Row] = {
+      generator: CodeGenerator,
+      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+      inputType: RelDataType,
+      inputTypeInfo: TypeInformation[Row],
+      inputFieldTypeInfo: Seq[TypeInformation[_]],
+      isRowTimeType: Boolean,
+      isPartitioned: Boolean,
+      isRowsClause: Boolean)
+    : ProcessFunction[Row, Row] = {
 
     val needRetract = false
     val (aggFields, aggregates) =
       transformToAggregateFunctions(
         namedAggregates.map(_.getKey),
         inputType,
-        needRetract)
+        needRetraction = false)
 
     val aggregationStateType: RowTypeInfo =
       createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
@@ -92,7 +98,7 @@ object AggregateUtil {
     val genFunction = generator.generateAggregations(
       "UnboundedProcessingOverAggregateHelper",
       generator,
-      inputType,
+      inputFieldTypeInfo,
       aggregates,
       aggFields,
       aggMapping,
@@ -112,13 +118,13 @@ object AggregateUtil {
         new RowTimeUnboundedRowsOver(
           genFunction,
           aggregationStateType,
-          FlinkTypeFactory.toInternalRowTypeInfo(inputType))
+          inputTypeInfo)
       } else {
         // RANGE unbounded over process function
         new RowTimeUnboundedRangeOver(
           genFunction,
           aggregationStateType,
-          FlinkTypeFactory.toInternalRowTypeInfo(inputType))
+          inputTypeInfo)
       }
     } else {
       if (isPartitioned) {
@@ -138,20 +144,25 @@ object AggregateUtil {
     * bounded OVER window to evaluate final aggregate value.
     *
     * @param generator       code generator instance
-    * @param namedAggregates List of calls to aggregate functions and their output field names
-    * @param inputType       Input row type
+    * @param namedAggregates Physical calls to aggregate functions and their output field names
+    * @param inputType Physical type of the row.
+    * @param inputTypeInfo Physical type information of the row.
+    * @param inputFieldTypeInfo Physical type information of the row's fields.
     * @param precedingOffset the preceding offset
     * @param isRowsClause   It is a tag that indicates whether the OVER clause is ROWS clause
     * @param isRowTimeType   It is a tag that indicates whether the time type is rowTimeType
     * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
     */
   private[flink] def createBoundedOverProcessFunction(
-    generator: CodeGenerator,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    precedingOffset: Long,
-    isRowsClause: Boolean,
-    isRowTimeType: Boolean): ProcessFunction[Row, Row] = {
+      generator: CodeGenerator,
+      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+      inputType: RelDataType,
+      inputTypeInfo: TypeInformation[Row],
+      inputFieldTypeInfo: Seq[TypeInformation[_]],
+      precedingOffset: Long,
+      isRowsClause: Boolean,
+      isRowTimeType: Boolean)
+    : ProcessFunction[Row, Row] = {
 
     val needRetract = true
     val (aggFields, aggregates) =
@@ -161,7 +172,6 @@ object AggregateUtil {
         needRetract)
 
     val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
-    val inputRowType = FlinkTypeFactory.toInternalRowTypeInfo(inputType).asInstanceOf[RowTypeInfo]
 
     val forwardMapping = (0 until inputType.getFieldCount).toArray
     val aggMapping = aggregates.indices.map(x => x + inputType.getFieldCount).toArray
@@ -170,7 +180,7 @@ object AggregateUtil {
     val genFunction = generator.generateAggregations(
       "BoundedOverAggregateHelper",
       generator,
-      inputType,
+      inputFieldTypeInfo,
       aggregates,
       aggFields,
       aggMapping,
@@ -189,14 +199,14 @@ object AggregateUtil {
         new RowTimeBoundedRowsOver(
           genFunction,
           aggregationStateType,
-          inputRowType,
+          inputTypeInfo,
           precedingOffset
         )
       } else {
         new RowTimeBoundedRangeOver(
           genFunction,
           aggregationStateType,
-          inputRowType,
+          inputTypeInfo,
           precedingOffset
         )
       }
@@ -206,13 +216,13 @@ object AggregateUtil {
           genFunction,
           precedingOffset,
           aggregationStateType,
-          inputRowType)
+          inputTypeInfo)
       } else {
         new ProcTimeBoundedRangeOver(
           genFunction,
           precedingOffset,
           aggregationStateType,
-          inputRowType)
+          inputTypeInfo)
       }
     }
   }
@@ -241,12 +251,13 @@ object AggregateUtil {
     * NOTE: this function is only used for time based window on batch tables.
     */
   def createDataSetWindowPrepareMapFunction(
-      generator: CodeGenerator,
-      window: LogicalWindow,
-      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-      groupings: Array[Int],
-      inputType: RelDataType,
-      isParserCaseSensitive: Boolean)
+    generator: CodeGenerator,
+    window: LogicalWindow,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    groupings: Array[Int],
+    inputType: RelDataType,
+    inputFieldTypeInfo: Seq[TypeInformation[_]],
+    isParserCaseSensitive: Boolean)
   : MapFunction[Row, Row] = {
 
     val needRetract = false
@@ -263,28 +274,28 @@ object AggregateUtil {
         Some(Array(BasicTypeInfo.LONG_TYPE_INFO)))
 
     val (timeFieldPos, tumbleTimeWindowSize) = window match {
-      case EventTimeTumblingGroupWindow(_, time, size) if isTimeInterval(size.resultType) =>
-        val timeFieldPos = getTimeFieldPosition(time, inputType, isParserCaseSensitive)
-        (timeFieldPos, Some(asLong(size)))
 
-      case EventTimeTumblingGroupWindow(_, time, _) =>
+      case TumblingGroupWindow(_, time, size) =>
         val timeFieldPos = getTimeFieldPosition(time, inputType, isParserCaseSensitive)
-        (timeFieldPos, None)
+        size match {
+          case Literal(value: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+            (timeFieldPos, Some(value))
+          case _ => (timeFieldPos, None)
+        }
 
-      case EventTimeSessionGroupWindow(_, time, _) =>
-        val timeFieldPos = getTimeFieldPosition(time, inputType, isParserCaseSensitive)
-        (timeFieldPos, None)
+      case SessionGroupWindow(_, time, _) =>
+        (getTimeFieldPosition(time, inputType, isParserCaseSensitive), None)
 
-      case EventTimeSlidingGroupWindow(_, time, size, slide)
-          if isTimeInterval(time.resultType) && doAllSupportPartialMerge(aggregates) =>
-        // pre-tumble incremental aggregates on time-windows
+      case SlidingGroupWindow(_, time, size, slide) =>
         val timeFieldPos = getTimeFieldPosition(time, inputType, isParserCaseSensitive)
-        val preTumblingSize = determineLargestTumblingSize(asLong(size), asLong(slide))
-        (timeFieldPos, Some(preTumblingSize))
-
-      case EventTimeSlidingGroupWindow(_, time, _, _) =>
-        val timeFieldPos = getTimeFieldPosition(time, inputType, isParserCaseSensitive)
-        (timeFieldPos, None)
+        size match {
+          case Literal(value: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+            // pre-tumble incremental aggregates on time-windows
+            val timeFieldPos = getTimeFieldPosition(time, inputType, isParserCaseSensitive)
+            val preTumblingSize = determineLargestTumblingSize(asLong(size), asLong(slide))
+            (timeFieldPos, Some(preTumblingSize))
+          case _ => (timeFieldPos, None)
+        }
 
       case _ =>
         throw new UnsupportedOperationException(s"$window is currently not supported on batch")
@@ -296,7 +307,7 @@ object AggregateUtil {
     val genFunction = generator.generateAggregations(
       "DataSetAggregatePrepareMapHelper",
       generator,
-      inputType,
+      inputFieldTypeInfo,
       aggregates,
       aggFieldIndexes,
       aggMapping,
@@ -349,31 +360,32 @@ object AggregateUtil {
       window: LogicalWindow,
       namedAggregates: Seq[CalcitePair[AggregateCall, String]],
       groupings: Array[Int],
-      inputType: RelDataType,
+      physicalInputRowType: RelDataType,
+      physicalInputTypes: Seq[TypeInformation[_]],
       isParserCaseSensitive: Boolean)
     : RichGroupReduceFunction[Row, Row] = {
 
     val needRetract = false
     val (aggFieldIndexes, aggregates) = transformToAggregateFunctions(
       namedAggregates.map(_.getKey),
-      inputType,
+      physicalInputRowType,
       needRetract)
 
     val returnType: RowTypeInfo = createDataSetAggregateBufferDataType(
       groupings,
       aggregates,
-      inputType,
+      physicalInputRowType,
       Some(Array(BasicTypeInfo.LONG_TYPE_INFO)))
 
     val keysAndAggregatesArity = groupings.length + namedAggregates.length
 
     window match {
-      case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
+      case SlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
         // sliding time-window for partial aggregations
         val genFunction = generator.generateAggregations(
           "DataSetAggregatePrepareMapHelper",
           generator,
-          inputType,
+          physicalInputTypes,
           aggregates,
           aggFieldIndexes,
           aggregates.indices.map(_ + groupings.length).toArray,
@@ -433,7 +445,7 @@ object AggregateUtil {
     : FlatMapFunction[Row, Row] = {
 
     window match {
-      case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
+      case SlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
         new DataSetSlideTimeWindowAggFlatMapFunction(
           inputType.getArity - 1,
           asLong(size),
@@ -458,7 +470,8 @@ object AggregateUtil {
       generator: CodeGenerator,
       window: LogicalWindow,
       namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-      inputType: RelDataType,
+      physicalInputRowType: RelDataType,
+      physicalInputTypes: Seq[TypeInformation[_]],
       outputType: RelDataType,
       groupings: Array[Int],
       properties: Seq[NamedWindowProperty],
@@ -468,7 +481,7 @@ object AggregateUtil {
     val needRetract = false
     val (aggFieldIndexes, aggregates) = transformToAggregateFunctions(
       namedAggregates.map(_.getKey),
-      inputType,
+      physicalInputRowType,
       needRetract)
 
     val aggMapping = aggregates.indices.toArray.map(_ + groupings.length)
@@ -476,7 +489,7 @@ object AggregateUtil {
     val genPreAggFunction = generator.generateAggregations(
       "GroupingWindowAggregateHelper",
       generator,
-      inputType,
+      physicalInputTypes,
       aggregates,
       aggFieldIndexes,
       aggMapping,
@@ -493,7 +506,7 @@ object AggregateUtil {
     val genFinalAggFunction = generator.generateAggregations(
       "GroupingWindowAggregateHelper",
       generator,
-      inputType,
+      physicalInputTypes,
       aggregates,
       aggFieldIndexes,
       aggMapping,
@@ -510,7 +523,7 @@ object AggregateUtil {
     val keysAndAggregatesArity = groupings.length + namedAggregates.length
 
     window match {
-      case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
+      case TumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
         // tumbling time window
         val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
         if (doAllSupportPartialMerge(aggregates)) {
@@ -532,13 +545,13 @@ object AggregateUtil {
             endPos,
             outputType.getFieldCount)
         }
-      case EventTimeTumblingGroupWindow(_, _, size) =>
+      case TumblingGroupWindow(_, _, size) =>
         // tumbling count window
         new DataSetTumbleCountWindowAggReduceGroupFunction(
           genFinalAggFunction,
           asLong(size))
 
-      case EventTimeSessionGroupWindow(_, _, gap) =>
+      case SessionGroupWindow(_, _, gap) =>
         val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
         new DataSetSessionWindowAggReduceGroupFunction(
           genFinalAggFunction,
@@ -548,7 +561,7 @@ object AggregateUtil {
           asLong(gap),
           isInputCombined)
 
-      case EventTimeSlidingGroupWindow(_, _, size, _) if isTimeInterval(size.resultType) =>
+      case SlidingGroupWindow(_, _, size, _) if isTimeInterval(size.resultType) =>
         val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
         if (doAllSupportPartialMerge(aggregates)) {
           // for partial aggregations
@@ -570,7 +583,7 @@ object AggregateUtil {
             asLong(size))
         }
 
-      case EventTimeSlidingGroupWindow(_, _, size, _) =>
+      case SlidingGroupWindow(_, _, size, _) =>
         new DataSetSlideWindowAggReduceGroupFunction(
             genFinalAggFunction,
             keysAndAggregatesArity,
@@ -608,13 +621,14 @@ object AggregateUtil {
     generator: CodeGenerator,
     window: LogicalWindow,
     namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
+    physicalInputRowType: RelDataType,
+    physicalInputTypes: Seq[TypeInformation[_]],
     groupings: Array[Int]): MapPartitionFunction[Row, Row] = {
 
     val needRetract = false
     val (aggFieldIndexes, aggregates) = transformToAggregateFunctions(
       namedAggregates.map(_.getKey),
-      inputType,
+      physicalInputRowType,
       needRetract)
 
     val aggMapping = aggregates.indices.map(_ + groupings.length).toArray
@@ -622,18 +636,18 @@ object AggregateUtil {
     val keysAndAggregatesArity = groupings.length + namedAggregates.length
 
     window match {
-      case EventTimeSessionGroupWindow(_, _, gap) =>
+      case SessionGroupWindow(_, _, gap) =>
         val combineReturnType: RowTypeInfo =
           createDataSetAggregateBufferDataType(
             groupings,
             aggregates,
-            inputType,
+            physicalInputRowType,
             Option(Array(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO)))
 
         val genFunction = generator.generateAggregations(
           "GroupingWindowAggregateHelper",
           generator,
-          inputType,
+          physicalInputTypes,
           aggregates,
           aggFieldIndexes,
           aggMapping,
@@ -679,14 +693,15 @@ object AggregateUtil {
       generator: CodeGenerator,
       window: LogicalWindow,
       namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-      inputType: RelDataType,
+      physicalInputRowType: RelDataType,
+      physicalInputTypes: Seq[TypeInformation[_]],
       groupings: Array[Int])
     : GroupCombineFunction[Row, Row] = {
 
     val needRetract = false
     val (aggFieldIndexes, aggregates) = transformToAggregateFunctions(
       namedAggregates.map(_.getKey),
-      inputType,
+      physicalInputRowType,
       needRetract)
 
     val aggMapping = aggregates.indices.map(_ + groupings.length).toArray
@@ -695,18 +710,18 @@ object AggregateUtil {
 
     window match {
 
-      case EventTimeSessionGroupWindow(_, _, gap) =>
+      case SessionGroupWindow(_, _, gap) =>
         val combineReturnType: RowTypeInfo =
           createDataSetAggregateBufferDataType(
             groupings,
             aggregates,
-            inputType,
+            physicalInputRowType,
             Option(Array(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO)))
 
         val genFunction = generator.generateAggregations(
           "GroupingWindowAggregateHelper",
           generator,
-          inputType,
+          physicalInputTypes,
           aggregates,
           aggFieldIndexes,
           aggMapping,
@@ -742,6 +757,7 @@ object AggregateUtil {
       generator: CodeGenerator,
       namedAggregates: Seq[CalcitePair[AggregateCall, String]],
       inputType: RelDataType,
+      inputFieldTypeInfo: Seq[TypeInformation[_]],
       outputType: RelDataType,
       groupings: Array[Int],
       inGroupingSet: Boolean): (Option[DataSetPreAggFunction],
@@ -786,7 +802,7 @@ object AggregateUtil {
       val genPreAggFunction = generator.generateAggregations(
         "DataSetAggregatePrepareMapHelper",
         generator,
-        inputType,
+        inputFieldTypeInfo,
         aggregates,
         aggInFields,
         aggregates.indices.map(_ + groupings.length).toArray,
@@ -813,7 +829,7 @@ object AggregateUtil {
       val genFinalAggFunction = generator.generateAggregations(
         "DataSetAggregateFinalHelper",
         generator,
-        inputType,
+        inputFieldTypeInfo,
         aggregates,
         aggInFields,
         aggOutFields,
@@ -837,7 +853,7 @@ object AggregateUtil {
       val genFunction = generator.generateAggregations(
         "DataSetAggregateHelper",
         generator,
-        inputType,
+        inputFieldTypeInfo,
         aggregates,
         aggInFields,
         aggOutFields,
@@ -914,6 +930,7 @@ object AggregateUtil {
       generator: CodeGenerator,
       namedAggregates: Seq[CalcitePair[AggregateCall, String]],
       inputType: RelDataType,
+      inputFieldTypeInfo: Seq[TypeInformation[_]],
       outputType: RelDataType,
       needMerge: Boolean)
     : (DataStreamAggFunction[Row, Row, Row], RowTypeInfo, RowTypeInfo) = {
@@ -931,7 +948,7 @@ object AggregateUtil {
     val genFunction = generator.generateAggregations(
       "GroupingWindowAggregateHelper",
       generator,
-      inputType,
+      inputFieldTypeInfo,
       aggregates,
       aggFields,
       aggMapping,
@@ -1047,12 +1064,9 @@ object AggregateUtil {
 
   private def isTimeWindow(window: LogicalWindow) = {
     window match {
-      case ProcessingTimeTumblingGroupWindow(_, size) => isTimeInterval(size.resultType)
-      case ProcessingTimeSlidingGroupWindow(_, size, _) => isTimeInterval(size.resultType)
-      case ProcessingTimeSessionGroupWindow(_, _) => true
-      case EventTimeTumblingGroupWindow(_, _, size) => isTimeInterval(size.resultType)
-      case EventTimeSlidingGroupWindow(_, _, size, _) => isTimeInterval(size.resultType)
-      case EventTimeSessionGroupWindow(_, _, _) => true
+      case TumblingGroupWindow(_, _, size) => isTimeIntervalLiteral(size)
+      case SlidingGroupWindow(_, _, size, _) => isTimeIntervalLiteral(size)
+      case SessionGroupWindow(_, _, _) => true
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
index 03ca02c..ef97e71 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
@@ -21,7 +21,7 @@ import java.util.{List => JList, ArrayList => JArrayList}
 
 import org.apache.flink.api.common.state._
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.table.codegen.{GeneratedAggregationsFunction, Compiler}
@@ -39,8 +39,8 @@ import org.slf4j.LoggerFactory
  */
 class RowTimeBoundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
-    aggregationStateType: RowTypeInfo,
-    inputRowType: RowTypeInfo,
+    aggregationStateType: TypeInformation[Row],
+    inputRowType: TypeInformation[Row],
     precedingOffset: Long)
   extends ProcessFunction[Row, Row]
     with Compiler[GeneratedAggregations] {

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
index 4a9a14c..7169cf7 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory
 class RowTimeBoundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     aggregationStateType: RowTypeInfo,
-    inputRowType: RowTypeInfo,
+    inputRowType: TypeInformation[Row],
     precedingOffset: Long)
   extends ProcessFunction[Row, Row]
     with Compiler[GeneratedAggregations] {

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
new file mode 100644
index 0000000..8466cdf
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
@@ -0,0 +1,47 @@
+/*
+ * 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.table.sources
+
+import org.apache.flink.api.java.tuple.Tuple2
+
+/**
+  * Defines logical time attributes for a [[TableSource]]. Time attributes can be used for
+  * indicating, accessing, and working with Flink's event-time or processing-time. A
+  * [[TableSource]] that implements this interface can define names and positions of rowtime
+  * and proctime attributes in the rows it produces.
+  */
+trait DefinedTimeAttributes {
+
+  /**
+    * Defines a name and position (starting at 0) of rowtime attribute that represents Flink's
+    * event-time. Null if no rowtime should be available. If the position is within the arity of
+    * the result row, the logical attribute will overwrite the physical attribute. If the position
+    * is higher than the result row, the time attribute will be appended logically.
+    */
+  def getRowtimeAttribute: Tuple2[Int, String]
+
+  /**
+    * Defines a name and position (starting at 0) of proctime attribute that represents Flink's
+    * processing-time. Null if no proctime should be available. If the position is within the arity
+    * of the result row, the logical attribute will overwrite the physical attribute. If the
+    * position is higher than the result row, the time attribute will be appended logically.
+    */
+  def getProctimeAttribute: Tuple2[Int, String]
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala
new file mode 100644
index 0000000..31dcb5c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.table.typeutils
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeutils.base.{SqlTimestampComparator, SqlTimestampSerializer}
+
+/**
+  * Type information for indicating event or processing time. However, it behaves like a
+  * regular SQL timestamp.
+  */
+class TimeIndicatorTypeInfo(val isEventTime: Boolean)
+  extends SqlTimeTypeInfo[Timestamp](
+    classOf[Timestamp],
+    SqlTimestampSerializer.INSTANCE,
+    classOf[SqlTimestampComparator].asInstanceOf[Class[TypeComparator[Timestamp]]]) {
+
+  override def toString: String = s"TimeIndicatorTypeInfo"
+}
+
+object TimeIndicatorTypeInfo {
+
+  val ROWTIME_INDICATOR = new TimeIndicatorTypeInfo(true)
+  val PROCTIME_INDICATOR = new TimeIndicatorTypeInfo(false)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/495f104b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
index 40f0cf2..9896a8c 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.typeutils
 
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
 import org.apache.flink.api.common.typeinfo._
 import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
 import org.apache.flink.table.validate._
@@ -29,6 +29,7 @@ object TypeCheckUtils {
     * SQL type but NOT vice versa.
     */
   def isAdvanced(dataType: TypeInformation[_]): Boolean = dataType match {
+    case _: TimeIndicatorTypeInfo => false
     case _: BasicTypeInfo[_] => false
     case _: SqlTimeTypeInfo[_] => false
     case _: TimeIntervalTypeInfo[_] => false
@@ -64,6 +65,8 @@ object TypeCheckUtils {
 
   def isInteger(dataType: TypeInformation[_]): Boolean = dataType == INT_TYPE_INFO
 
+  def isLong(dataType: TypeInformation[_]): Boolean = dataType == LONG_TYPE_INFO
+
   def isArray(dataType: TypeInformation[_]): Boolean = dataType match {
     case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => true
     case _ => false