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/12 06:42:49 UTC

[1/5] flink git commit: [FLINK-6491] [table] Add QueryConfig and state clean up for over-windowed aggregates.

Repository: flink
Updated Branches:
  refs/heads/release-1.3 f3ce0885f -> 609d5a32f


http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 910cbf2..9da2c44 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
@@ -18,11 +18,12 @@
 
 package org.apache.flink.table.api.scala.stream.table
 
+import org.apache.flink.api.common.time.Time
 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.table.api.{StreamQueryConfig, TableEnvironment}
 import org.apache.flink.table.api.scala.stream.utils.StreamITCase.RetractingSink
 import org.apache.flink.types.Row
 import org.junit.Assert.assertEquals
@@ -34,6 +35,8 @@ import scala.collection.mutable
   * Tests of groupby (without window) aggregations
   */
 class GroupAggregationsITCase extends StreamingWithStateTestBase {
+  private val queryConfig = new StreamQueryConfig()
+  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
 
   @Test
   def testNonKeyedGroupAggregate(): Unit = {
@@ -45,7 +48,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
     val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
             .select('a.sum, 'b.sum)
 
-    val results = t.toRetractStream[Row]
+    val results = t.toRetractStream[Row](queryConfig)
     results.addSink(new StreamITCase.RetractingSink).setParallelism(1)
     env.execute()
 
@@ -64,7 +67,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
       .groupBy('b)
       .select('b, 'a.sum)
 
-    val results = t.toRetractStream[Row]
+    val results = t.toRetractStream[Row](queryConfig)
     results.addSink(new StreamITCase.RetractingSink)
     env.execute()
 
@@ -85,7 +88,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
       .groupBy('cnt)
       .select('cnt, 'b.count as 'freq)
 
-    val results = t.toRetractStream[Row]
+    val results = t.toRetractStream[Row](queryConfig)
 
     results.addSink(new RetractingSink)
     env.execute()
@@ -104,7 +107,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase {
       .groupBy('e, 'b % 3)
       .select('c.min, 'e, 'a.avg, 'd.count)
 
-    val results = t.toRetractStream[Row]
+    val results = t.toRetractStream[Row](queryConfig)
     results.addSink(new RetractingSink)
     env.execute()
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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
deleted file mode 100644
index eadcfc8..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala
+++ /dev/null
@@ -1,336 +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.runtime.aggregate
-
-import java.util.Comparator
-import java.util.concurrent.ConcurrentLinkedQueue
-import java.lang.{Integer => JInt, Long => JLong}
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.streaming.api.operators.KeyedProcessOperator
-import org.apache.flink.streaming.api.watermark.Watermark
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
-import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil}
-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
-
-class BoundedProcessingOverRangeProcessFunctionTest {
-
-  @Test
-  def testProcTimePartitionedOverRange(): Unit = {
-
-    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")))
-
-    val aggregates =
-      Array(new LongMinWithRetractAggFunction,
-            new LongMaxWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]]
-    val aggregationStateType: RowTypeInfo = AggregateUtil.createAccumulatorRowType(aggregates)
-
-    val funcCode =
-      """
-        |public class BoundedOverAggregateHelper$33
-        |  extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations {
-        |
-        |  transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction
-        |    fmin = null;
-        |
-        |  transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction
-        |    fmax = null;
-        |
-        |  public BoundedOverAggregateHelper$33() throws Exception {
-        |
-        |    fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction)
-        |    org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
-        |    .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" +
-        |    "MuTG9uZ01pbldpdGhSZXRyYWN0QWdnRnVuY3Rpb26oIdX_DaMPxQIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" +
-        |    "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWluV2l0aFJldHJhY3RBZ2dGdW5jdGlvbq_ZGuzxtA_S" +
-        |    "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" +
-        |    "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" +
-        |    "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" +
-        |    "mluZyRMb25nJOda0iCPo2ukAgAAeHA");
-        |
-        |    fmax = (org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction)
-        |    org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
-        |    .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" +
-        |    "MuTG9uZ01heFdpdGhSZXRyYWN0QWdnRnVuY3Rpb25RmsI8azNGXwIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" +
-        |    "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWF4V2l0aFJldHJhY3RBZ2dGdW5jdGlvbvnwowlX0_Qf" +
-        |    "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" +
-        |    "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" +
-        |    "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" +
-        |    "mluZyRMb25nJOda0iCPo2ukAgAAeHA");
-        |  }
-        |
-        |  public void setAggregationResults(
-        |    org.apache.flink.types.Row accs,
-        |    org.apache.flink.types.Row output) {
-        |
-        |    org.apache.flink.table.functions.AggregateFunction baseClass0 =
-        |      (org.apache.flink.table.functions.AggregateFunction) fmin;
-        |    output.setField(5, baseClass0.getValue(
-        |      (org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
-        |      accs.getField(0)));
-        |
-        |    org.apache.flink.table.functions.AggregateFunction baseClass1 =
-        |      (org.apache.flink.table.functions.AggregateFunction) fmax;
-        |    output.setField(6, baseClass1.getValue(
-        |      (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
-        |      accs.getField(1)));
-        |  }
-        |
-        |  public void accumulate(
-        |    org.apache.flink.types.Row accs,
-        |    org.apache.flink.types.Row input) {
-        |
-        |    fmin.accumulate(
-        |      ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
-        |      accs.getField(0)),
-        |      (java.lang.Long) input.getField(4));
-        |
-        |    fmax.accumulate(
-        |      ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
-        |      accs.getField(1)),
-        |      (java.lang.Long) input.getField(4));
-        |  }
-        |
-        |  public void retract(
-        |    org.apache.flink.types.Row accs,
-        |    org.apache.flink.types.Row input) {
-        |
-        |    fmin.retract(
-        |      ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
-        |      accs.getField(0)),
-        |      (java.lang.Long) input.getField(4));
-        |
-        |    fmax.retract(
-        |      ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
-        |      accs.getField(1)),
-        |      (java.lang.Long) input.getField(4));
-        |  }
-        |
-        |  public org.apache.flink.types.Row createAccumulators() {
-        |
-        |    org.apache.flink.types.Row accs = new org.apache.flink.types.Row(2);
-        |
-        |    accs.setField(
-        |      0,
-        |      fmin.createAccumulator());
-        |
-        |    accs.setField(
-        |      1,
-        |      fmax.createAccumulator());
-        |
-        |      return accs;
-        |  }
-        |
-        |  public void setForwardedFields(
-        |    org.apache.flink.types.Row input,
-        |    org.apache.flink.types.Row output) {
-        |
-        |    output.setField(0, input.getField(0));
-        |    output.setField(1, input.getField(1));
-        |    output.setField(2, input.getField(2));
-        |    output.setField(3, input.getField(3));
-        |    output.setField(4, input.getField(4));
-        |  }
-        |
-        |  public org.apache.flink.types.Row createOutputRow() {
-        |    return new org.apache.flink.types.Row(7);
-        |  }
-        |
-        |/*******  This test does not use the following methods  *******/
-        |  public org.apache.flink.types.Row mergeAccumulatorsPair(
-        |    org.apache.flink.types.Row a,
-        |    org.apache.flink.types.Row b) {
-        |    return null;
-        |  }
-        |
-        |  public void resetAccumulator(org.apache.flink.types.Row accs) {
-        |  }
-        |
-        |  public void setConstantFlags(org.apache.flink.types.Row output) {
-        |  }
-        |}
-      """.stripMargin
-
-    val funcName = "BoundedOverAggregateHelper$33"
-
-    val genAggFunction = GeneratedAggregationsFunction(funcName, funcCode)
-    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
-      new ProcTimeBoundedRangeOver(
-        genAggFunction,
-        1000,
-        aggregationStateType,
-        rT))
-
-    val testHarness = new KeyedOneInputStreamOperatorTestHarness[JInt, CRow, CRow](
-      processFunction,
-      new TupleRowSelector(0),
-      BasicTypeInfo.INT_TYPE_INFO)
-
-    testHarness.open()
-
-    // Time = 3
-    testHarness.setProcessingTime(3)
-    // key = 1
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0))
-    // key = 2
-    testHarness.processElement(new StreamRecord(
-      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(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0))
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 0))
-    // key = 2
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 0))
-
-    // Time = 5
-    testHarness.setProcessingTime(5)
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0))
-
-    // Time = 6
-    testHarness.setProcessingTime(6)
-
-    // Time = 1002
-    testHarness.setProcessingTime(1002)
-    // key = 1
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0))
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0))
-    // key = 2
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0))
-
-    // Time = 1003
-    testHarness.setProcessingTime(1003)
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0))
-
-    // Time = 1004
-    testHarness.setProcessingTime(1004)
-    testHarness.processElement(new StreamRecord(
-      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(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 0))
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 0))
-    // key = 2
-    testHarness.processElement(new StreamRecord(
-      new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 0))
-
-    testHarness.setProcessingTime(1006)
-
-    val result = testHarness.getOutput
-
-    val expectedOutput = new ConcurrentLinkedQueue[Object]()
-
-    // all elements at the same proc timestamp have the same value
-    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))
-
-    testHarness.close()
-
-  }
-}
-
-object BoundedProcessingOverRangeProcessFunctionTest {
-
-/**
- * Return 0 for equal CRows and non zero for different CRows
- */
-class RowResultSortComparator(indexCounter: Int) extends Comparator[Object] with Serializable {
-
-    override def compare(o1: Object, o2: Object):Int = {
-
-      if (o1.isInstanceOf[Watermark] || o2.isInstanceOf[Watermark]) {
-        // watermark is not expected
-         -1
-       } else {
-        val row1 = o1.asInstanceOf[StreamRecord[CRow]].getValue
-        val row2 = o2.asInstanceOf[StreamRecord[CRow]].getValue
-        row1.toString.compareTo(row2.toString)
-      }
-   }
-}
-
-/**
- * Simple test class that returns a specified field as the selector function
- */
-class TupleRowSelector(
-    private val selectorField:Int) extends KeySelector[CRow, Integer] {
-
-  override def getKey(value: CRow): Integer = {
-    value.row.getField(selectorField).asInstanceOf[Integer]
-  }
-}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
index eb5acd5b..77798f9 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
@@ -19,15 +19,294 @@ package org.apache.flink.table.runtime.harness
 
 import java.util.{Comparator, Queue => JQueue}
 
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator
 import org.apache.flink.streaming.api.watermark.Watermark
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
 import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil}
-import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.codegen.GeneratedAggregationsFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.flink.table.functions.aggfunctions.{LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction, IntSumWithRetractAggFunction}
+import org.apache.flink.table.runtime.aggregate.AggregateUtil
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 class HarnessTestBase {
+
+  protected val MinMaxRowType = 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"))
+
+  protected val SumRowType = new RowTypeInfo(Array[TypeInformation[_]](
+    LONG_TYPE_INFO,
+    INT_TYPE_INFO,
+    STRING_TYPE_INFO),
+    Array("a", "b", "c"))
+
+  protected val minMaxCRowType = new CRowTypeInfo(MinMaxRowType)
+  protected val sumCRowType = new CRowTypeInfo(SumRowType)
+
+  protected val minMaxAggregates =
+    Array(new LongMinWithRetractAggFunction,
+          new LongMaxWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]]
+
+  protected val sumAggregates =
+    Array(new IntSumWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]]
+
+  protected val minMaxAggregationStateType: RowTypeInfo =
+    AggregateUtil.createAccumulatorRowType(minMaxAggregates)
+
+  protected val sumAggregationStateType: RowTypeInfo =
+    AggregateUtil.createAccumulatorRowType(sumAggregates)
+
+  val minMaxCode: String =
+    """
+      |public class MinMaxAggregateHelper
+      |  extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations {
+      |
+      |  transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction
+      |    fmin = null;
+      |
+      |  transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction
+      |    fmax = null;
+      |
+      |  public MinMaxAggregateHelper() throws Exception {
+      |
+      |    fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction)
+      |    org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
+      |    .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" +
+      |    "MuTG9uZ01pbldpdGhSZXRyYWN0QWdnRnVuY3Rpb26oIdX_DaMPxQIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" +
+      |    "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWluV2l0aFJldHJhY3RBZ2dGdW5jdGlvbq_ZGuzxtA_S" +
+      |    "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" +
+      |    "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" +
+      |    "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" +
+      |    "mluZyRMb25nJOda0iCPo2ukAgAAeHA");
+      |
+      |    fmax = (org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction)
+      |    org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
+      |    .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" +
+      |    "MuTG9uZ01heFdpdGhSZXRyYWN0QWdnRnVuY3Rpb25RmsI8azNGXwIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" +
+      |    "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWF4V2l0aFJldHJhY3RBZ2dGdW5jdGlvbvnwowlX0_Qf" +
+      |    "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" +
+      |    "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" +
+      |    "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" +
+      |    "mluZyRMb25nJOda0iCPo2ukAgAAeHA");
+      |  }
+      |
+      |  public void setAggregationResults(
+      |    org.apache.flink.types.Row accs,
+      |    org.apache.flink.types.Row output) {
+      |
+      |    org.apache.flink.table.functions.AggregateFunction baseClass0 =
+      |      (org.apache.flink.table.functions.AggregateFunction) fmin;
+      |    output.setField(5, baseClass0.getValue(
+      |      (org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
+      |      accs.getField(0)));
+      |
+      |    org.apache.flink.table.functions.AggregateFunction baseClass1 =
+      |      (org.apache.flink.table.functions.AggregateFunction) fmax;
+      |    output.setField(6, baseClass1.getValue(
+      |      (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
+      |      accs.getField(1)));
+      |  }
+      |
+      |  public void accumulate(
+      |    org.apache.flink.types.Row accs,
+      |    org.apache.flink.types.Row input) {
+      |
+      |    fmin.accumulate(
+      |      ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
+      |      accs.getField(0)),
+      |      (java.lang.Long) input.getField(4));
+      |
+      |    fmax.accumulate(
+      |      ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
+      |      accs.getField(1)),
+      |      (java.lang.Long) input.getField(4));
+      |  }
+      |
+      |  public void retract(
+      |    org.apache.flink.types.Row accs,
+      |    org.apache.flink.types.Row input) {
+      |
+      |    fmin.retract(
+      |      ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
+      |      accs.getField(0)),
+      |      (java.lang.Long) input.getField(4));
+      |
+      |    fmax.retract(
+      |      ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
+      |      accs.getField(1)),
+      |      (java.lang.Long) input.getField(4));
+      |  }
+      |
+      |  public org.apache.flink.types.Row createAccumulators() {
+      |
+      |    org.apache.flink.types.Row accs = new org.apache.flink.types.Row(2);
+      |
+      |    accs.setField(
+      |      0,
+      |      fmin.createAccumulator());
+      |
+      |    accs.setField(
+      |      1,
+      |      fmax.createAccumulator());
+      |
+      |      return accs;
+      |  }
+      |
+      |  public void setForwardedFields(
+      |    org.apache.flink.types.Row input,
+      |    org.apache.flink.types.Row output) {
+      |
+      |    output.setField(0, input.getField(0));
+      |    output.setField(1, input.getField(1));
+      |    output.setField(2, input.getField(2));
+      |    output.setField(3, input.getField(3));
+      |    output.setField(4, input.getField(4));
+      |  }
+      |
+      |  public org.apache.flink.types.Row createOutputRow() {
+      |    return new org.apache.flink.types.Row(7);
+      |  }
+      |
+      |/*******  This test does not use the following methods  *******/
+      |  public org.apache.flink.types.Row mergeAccumulatorsPair(
+      |    org.apache.flink.types.Row a,
+      |    org.apache.flink.types.Row b) {
+      |    return null;
+      |  }
+      |
+      |  public void resetAccumulator(org.apache.flink.types.Row accs) {
+      |  }
+      |
+      |  public void setConstantFlags(org.apache.flink.types.Row output) {
+      |  }
+      |}
+    """.stripMargin
+
+  val sumAggCode: String =
+    """
+      |public final class SumAggregationHelper
+      |  extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations {
+      |
+      |
+      |transient org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction
+      |sum = null;
+      |private final org.apache.flink.table.runtime.aggregate.SingleElementIterable<org.apache
+      |    .flink.table.functions.aggfunctions.SumWithRetractAccumulator> accIt0 =
+      |      new org.apache.flink.table.runtime.aggregate.SingleElementIterable<org.apache.flink
+      |      .table
+      |      .functions.aggfunctions.SumWithRetractAccumulator>();
+      |
+      |  public SumAggregationHelper() throws Exception {
+      |
+      |sum = (org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction)
+      |org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
+      |.deserialize
+      |("rO0ABXNyAEpvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuSW50U3VtV2l0a" +
+      |"FJldHJhY3RBZ2dGdW5jdGlvblkfWkeNZDeDAgAAeHIAR29yZy5hcGFjaGUuZmxpbmsudGFibGUuZnVuY3Rpb25" +
+      |"zLmFnZ2Z1bmN0aW9ucy5TdW1XaXRoUmV0cmFjdEFnZ0Z1bmN0aW9ut2oWrOsLrs0CAAFMAAdudW1lcmljdAAUT" +
+      |"HNjYWxhL21hdGgvTnVtZXJpYzt4cgAyb3JnLmFwYWNoZS5mbGluay50YWJsZS5mdW5jdGlvbnMuQWdncmVnYXR" +
+      |"lRnVuY3Rpb25NxhU-0mM1_AIAAHhyADRvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5Vc2VyRGVma" +
+      |"W5lZEZ1bmN0aW9uLQH3VDG4DJMCAAB4cHNyACFzY2FsYS5tYXRoLk51bWVyaWMkSW50SXNJbnRlZ3JhbCTw6XA" +
+      |"59sPAzAIAAHhw");
+      |
+      |
+      |  }
+      |
+      |  public final void setAggregationResults(
+      |    org.apache.flink.types.Row accs,
+      |    org.apache.flink.types.Row output) {
+      |
+      |    org.apache.flink.table.functions.AggregateFunction baseClass0 =
+      |      (org.apache.flink.table.functions.AggregateFunction)
+      |      sum;
+      |
+      |    output.setField(
+      |      1,
+      |      baseClass0.getValue((org.apache.flink.table.functions.aggfunctions
+      |      .SumWithRetractAccumulator) accs.getField(0)));
+      |  }
+      |
+      |  public final void accumulate(
+      |    org.apache.flink.types.Row accs,
+      |    org.apache.flink.types.Row input) {
+      |
+      |    sum.accumulate(
+      |      ((org.apache.flink.table.functions.aggfunctions.SumWithRetractAccumulator) accs
+      |      .getField
+      |      (0)),
+      |      (java.lang.Integer) input.getField(1));
+      |  }
+      |
+      |
+      |  public final void retract(
+      |    org.apache.flink.types.Row accs,
+      |    org.apache.flink.types.Row input) {
+      |  }
+      |
+      |  public final org.apache.flink.types.Row createAccumulators()
+      |     {
+      |
+      |      org.apache.flink.types.Row accs =
+      |          new org.apache.flink.types.Row(1);
+      |
+      |    accs.setField(
+      |      0,
+      |      sum.createAccumulator());
+      |
+      |      return accs;
+      |  }
+      |
+      |  public final void setForwardedFields(
+      |    org.apache.flink.types.Row input,
+      |    org.apache.flink.types.Row output)
+      |     {
+      |
+      |    output.setField(
+      |      0,
+      |      input.getField(0));
+      |  }
+      |
+      |  public final void setConstantFlags(org.apache.flink.types.Row output)
+      |     {
+      |
+      |  }
+      |
+      |  public final org.apache.flink.types.Row createOutputRow() {
+      |    return new org.apache.flink.types.Row(2);
+      |  }
+      |
+      |
+      |  public final org.apache.flink.types.Row mergeAccumulatorsPair(
+      |    org.apache.flink.types.Row a,
+      |    org.apache.flink.types.Row b)
+      |            {
+      |
+      |      return a;
+      |
+      |  }
+      |
+      |  public final void resetAccumulator(
+      |    org.apache.flink.types.Row accs) {
+      |  }
+      |}
+      |""".stripMargin
+
+
+  protected val minMaxFuncName = "MinMaxAggregateHelper"
+  protected val sumFuncName = "SumAggregationHelper"
+
+  protected val genMinMaxAggFunction = GeneratedAggregationsFunction(minMaxFuncName, minMaxCode)
+  protected val genSumAggFunction = GeneratedAggregationsFunction(sumFuncName, sumAggCode)
+
   def createHarnessTester[IN, OUT, KEY](
     operator: OneInputStreamOperator[IN, OUT],
     keySelector: KeySelector[IN, KEY],

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
new file mode 100644
index 0000000..04214f9
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
@@ -0,0 +1,157 @@
+/*
+ * 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.harness
+
+import java.lang.{Integer => JInt, Long => JLong}
+import java.util.concurrent.ConcurrentLinkedQueue
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.table.api.StreamQueryConfig
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.runtime.harness.HarnessTestBase._
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.junit.Test
+
+class NonWindowHarnessTest extends HarnessTestBase {
+
+  protected var queryConfig =
+    new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3))
+
+  @Test
+  def testProcTimeNonWindow(): Unit = {
+
+    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
+      new GroupAggProcessFunction(
+        genSumAggFunction,
+        sumAggregationStateType,
+        false,
+        queryConfig))
+
+    val testHarness =
+      createHarnessTester(
+        processFunction,
+        new TupleRowKeySelector[String](2),
+        BasicTypeInfo.STRING_TYPE_INFO)
+
+    testHarness.open()
+
+    // register cleanup timer with 3001
+    testHarness.setProcessingTime(1)
+
+    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
+    // reuse timer 3001
+    testHarness.setProcessingTime(1000)
+    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
+
+    // register cleanup timer with 4002
+    testHarness.setProcessingTime(1002)
+    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
+
+    // trigger cleanup timer and register cleanup timer with 7003
+    testHarness.setProcessingTime(4003)
+    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
+
+    val result = testHarness.getOutput
+
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
+
+    verify(expectedOutput, result, new RowResultSortComparator(6))
+
+    testHarness.close()
+  }
+
+  @Test
+  def testProcTimeNonWindowWithRetract(): Unit = {
+
+    val processFunction = new KeyedProcessOperator[String, CRow, CRow](
+      new GroupAggProcessFunction(
+        genSumAggFunction,
+        sumAggregationStateType,
+        true,
+        queryConfig))
+
+    val testHarness =
+      createHarnessTester(
+        processFunction,
+        new TupleRowKeySelector[String](2),
+        BasicTypeInfo.STRING_TYPE_INFO)
+
+    testHarness.open()
+
+    // register cleanup timer with 3001
+    testHarness.setProcessingTime(1)
+
+    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 2))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 3))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "ccc"), true), 4))
+
+    // trigger cleanup timer and register cleanup timer with 6002
+    testHarness.setProcessingTime(3002)
+    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 5))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 6))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 7))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "eee"), true), 8))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 9))
+    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 10))
+
+    val result = testHarness.getOutput
+
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 2))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 1: JInt), false), 3))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 3))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt), true), 4))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt), true), 5))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt), true), 6))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 4: JInt), false), 7))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 9: JInt), true), 7))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt), true), 8))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 9: JInt), false), 9))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 16: JInt), true), 9))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 2: JInt), false), 10))
+    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 5: JInt), true), 10))
+
+    verify(expectedOutput, result, new RowResultSortComparator(0))
+
+    testHarness.close()
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
index 56ca85c..786a843 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
@@ -18,180 +18,34 @@
 package org.apache.flink.table.runtime.harness
 
 import java.lang.{Integer => JInt, Long => JLong}
-import java.util.concurrent.ConcurrentLinkedQueue
+import java.util.concurrent.{ConcurrentLinkedQueue}
 
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
 import org.apache.flink.streaming.api.operators.KeyedProcessOperator
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
-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.api.StreamQueryConfig
 import org.apache.flink.table.runtime.aggregate._
 import org.apache.flink.table.runtime.harness.HarnessTestBase._
-import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.runtime.types.CRow
 import org.apache.flink.types.Row
 import org.junit.Test
 
 class OverWindowHarnessTest extends HarnessTestBase{
 
-  private val rT = 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"))
-
-  private val cRT = new CRowTypeInfo(rT)
-
-  private val aggregates =
-    Array(new LongMinWithRetractAggFunction,
-      new LongMaxWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]]
-  private val aggregationStateType: RowTypeInfo = AggregateUtil.createAccumulatorRowType(aggregates)
-
-  val funcCode: String =
-    """
-      |public class BoundedOverAggregateHelper
-      |  extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations {
-      |
-      |  transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction
-      |    fmin = null;
-      |
-      |  transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction
-      |    fmax = null;
-      |
-      |  public BoundedOverAggregateHelper() throws Exception {
-      |
-      |    fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction)
-      |    org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
-      |    .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" +
-      |    "MuTG9uZ01pbldpdGhSZXRyYWN0QWdnRnVuY3Rpb26oIdX_DaMPxQIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" +
-      |    "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWluV2l0aFJldHJhY3RBZ2dGdW5jdGlvbq_ZGuzxtA_S" +
-      |    "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" +
-      |    "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" +
-      |    "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" +
-      |    "mluZyRMb25nJOda0iCPo2ukAgAAeHA");
-      |
-      |    fmax = (org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction)
-      |    org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
-      |    .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" +
-      |    "MuTG9uZ01heFdpdGhSZXRyYWN0QWdnRnVuY3Rpb25RmsI8azNGXwIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" +
-      |    "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWF4V2l0aFJldHJhY3RBZ2dGdW5jdGlvbvnwowlX0_Qf" +
-      |    "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" +
-      |    "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" +
-      |    "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" +
-      |    "mluZyRMb25nJOda0iCPo2ukAgAAeHA");
-      |  }
-      |
-      |  public void setAggregationResults(
-      |    org.apache.flink.types.Row accs,
-      |    org.apache.flink.types.Row output) {
-      |
-      |    org.apache.flink.table.functions.AggregateFunction baseClass0 =
-      |      (org.apache.flink.table.functions.AggregateFunction) fmin;
-      |    output.setField(5, baseClass0.getValue(
-      |      (org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
-      |      accs.getField(0)));
-      |
-      |    org.apache.flink.table.functions.AggregateFunction baseClass1 =
-      |      (org.apache.flink.table.functions.AggregateFunction) fmax;
-      |    output.setField(6, baseClass1.getValue(
-      |      (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
-      |      accs.getField(1)));
-      |  }
-      |
-      |  public void accumulate(
-      |    org.apache.flink.types.Row accs,
-      |    org.apache.flink.types.Row input) {
-      |
-      |    fmin.accumulate(
-      |      ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
-      |      accs.getField(0)),
-      |      (java.lang.Long) input.getField(4));
-      |
-      |    fmax.accumulate(
-      |      ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
-      |      accs.getField(1)),
-      |      (java.lang.Long) input.getField(4));
-      |  }
-      |
-      |  public void retract(
-      |    org.apache.flink.types.Row accs,
-      |    org.apache.flink.types.Row input) {
-      |
-      |    fmin.retract(
-      |      ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator)
-      |      accs.getField(0)),
-      |      (java.lang.Long) input.getField(4));
-      |
-      |    fmax.retract(
-      |      ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator)
-      |      accs.getField(1)),
-      |      (java.lang.Long) input.getField(4));
-      |  }
-      |
-      |  public org.apache.flink.types.Row createAccumulators() {
-      |
-      |    org.apache.flink.types.Row accs = new org.apache.flink.types.Row(2);
-      |
-      |    accs.setField(
-      |      0,
-      |      fmin.createAccumulator());
-      |
-      |    accs.setField(
-      |      1,
-      |      fmax.createAccumulator());
-      |
-      |      return accs;
-      |  }
-      |
-      |  public void setForwardedFields(
-      |    org.apache.flink.types.Row input,
-      |    org.apache.flink.types.Row output) {
-      |
-      |    output.setField(0, input.getField(0));
-      |    output.setField(1, input.getField(1));
-      |    output.setField(2, input.getField(2));
-      |    output.setField(3, input.getField(3));
-      |    output.setField(4, input.getField(4));
-      |  }
-      |
-      |  public org.apache.flink.types.Row createOutputRow() {
-      |    return new org.apache.flink.types.Row(7);
-      |  }
-      |
-      |/*******  This test does not use the following methods  *******/
-      |  public org.apache.flink.types.Row mergeAccumulatorsPair(
-      |    org.apache.flink.types.Row a,
-      |    org.apache.flink.types.Row b) {
-      |    return null;
-      |  }
-      |
-      |  public void resetAccumulator(org.apache.flink.types.Row accs) {
-      |  }
-      |
-      |  public void setConstantFlags(org.apache.flink.types.Row output) {
-      |  }
-      |}
-    """.stripMargin
-
-
-  private val funcName = "BoundedOverAggregateHelper"
-
-  private val genAggFunction = GeneratedAggregationsFunction(funcName, funcCode)
-
+  protected var queryConfig =
+    new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3))
 
   @Test
   def testProcTimeBoundedRowsOver(): Unit = {
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new ProcTimeBoundedRowsOver(
-        genAggFunction,
+        genMinMaxAggFunction,
         2,
-        aggregationStateType,
-        cRT))
+        minMaxAggregationStateType,
+        minMaxCRowType,
+        queryConfig))
 
     val testHarness =
       createHarnessTester(processFunction,new TupleRowKeySelector[Integer](0),BasicTypeInfo
@@ -199,6 +53,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.open()
 
+    // register cleanup timer with 3001
     testHarness.setProcessingTime(1)
 
     testHarness.processElement(new StreamRecord(
@@ -209,6 +64,9 @@ class OverWindowHarnessTest extends HarnessTestBase{
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 1))
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 1))
+
+    // register cleanup timer with 4100
+    testHarness.setProcessingTime(1100)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 1))
     testHarness.processElement(new StreamRecord(
@@ -220,15 +78,19 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 1))
 
-    testHarness.setProcessingTime(2)
+    // register cleanup timer with 6001
+    testHarness.setProcessingTime(3001)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 2))
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 2))
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 2))
+
+    // trigger cleanup timer and register cleanup timer with 9002
+    testHarness.setProcessingTime(6002)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 2))
+        CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 2))
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 2))
 
@@ -274,10 +136,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
         Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 9L: JLong), true), 2))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 2))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 2))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), true), 2))
+        Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 2))
 
     verify(expectedOutput, result, new RowResultSortComparator(6))
 
@@ -292,10 +154,11 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new ProcTimeBoundedRangeOver(
-        genAggFunction,
-        1000,
-        aggregationStateType,
-        cRT))
+        genMinMaxAggFunction,
+        4000,
+        minMaxAggregationStateType,
+        minMaxCRowType,
+        queryConfig))
 
     val testHarness =
       createHarnessTester(
@@ -305,6 +168,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.open()
 
+    // register cleanup timer with 3003
     testHarness.setProcessingTime(3)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0))
@@ -314,6 +178,9 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.setProcessingTime(4)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0))
+
+    // trigger cleanup timer and register cleanup timer with 6003
+    testHarness.setProcessingTime(3003)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
@@ -323,9 +190,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0))
 
-    testHarness.setProcessingTime(6)
+    // register cleanup timer with 9002
+    testHarness.setProcessingTime(6002)
 
-    testHarness.setProcessingTime(1002)
+    testHarness.setProcessingTime(7002)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
@@ -333,15 +201,15 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0))
 
-    testHarness.setProcessingTime(1003)
+    // register cleanup timer with 14002
+    testHarness.setProcessingTime(11002)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0))
 
-    testHarness.setProcessingTime(1004)
+    testHarness.setProcessingTime(11004)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0))
 
-    testHarness.setProcessingTime(1005)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
@@ -349,7 +217,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 0))
 
-    testHarness.setProcessingTime(1006)
+    testHarness.setProcessingTime(11006)
 
     val result = testHarness.getOutput
 
@@ -364,40 +232,40 @@ class OverWindowHarnessTest extends HarnessTestBase{
         Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 4))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 5))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 5))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 5))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 3L: JLong, 4L: JLong), true), 3004))
     expectedOutput.add(new StreamRecord(
-      CRow(
-        Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 5))
+      CRow(Row.of(
+        2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 20L: JLong, 20L: JLong), true), 3004))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 6))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 4L: JLong, 4L: JLong), true), 6))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 1003))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 5L: JLong, 6L: JLong), true), 7003))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 1003))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true), 7003))
     expectedOutput.add(new StreamRecord(
       CRow(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 1003))
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 7003))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 1004))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 11003))
     expectedOutput.add(new StreamRecord(
-      CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 1005))
+      CRow(Row.of(
+        1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 10L: JLong), true), 11005))
     expectedOutput.add(new StreamRecord(
-      CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 4L: JLong, 10L: JLong), true), 1006))
+      CRow(Row.of(
+        1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 10L: JLong), true), 11005))
     expectedOutput.add(new StreamRecord(
       CRow(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 4L: JLong, 10L: JLong), true), 1006))
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 7L: JLong, 10L: JLong), true), 11005))
     expectedOutput.add(new StreamRecord(
       CRow(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), true), 1006))
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 11005))
 
     verify(expectedOutput, result, new RowResultSortComparator(6))
 
@@ -409,8 +277,9 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new ProcTimeUnboundedPartitionedOver(
-        genAggFunction,
-        aggregationStateType))
+        genMinMaxAggFunction,
+        minMaxAggregationStateType,
+        queryConfig))
 
     val testHarness =
       createHarnessTester(
@@ -420,6 +289,9 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.open()
 
+    // register cleanup timer with 4003
+    testHarness.setProcessingTime(1003)
+
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
@@ -438,18 +310,19 @@ class OverWindowHarnessTest extends HarnessTestBase{
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0))
-
-    testHarness.setProcessingTime(1003)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 1003))
+      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 1003))
+      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0))
+
+    // trigger cleanup timer and register cleanup timer with 8003
+    testHarness.setProcessingTime(5003)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 1003))
+      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 5003))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 1003))
+      CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 5003))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 1003))
+      CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 5003))
 
     val result = testHarness.getOutput
 
@@ -484,19 +357,19 @@ class OverWindowHarnessTest extends HarnessTestBase{
         Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 0))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 1003))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 0))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 1003))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 0))
     expectedOutput.add(new StreamRecord(
       CRow(
-        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true), 1003))
+        Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 5003))
     expectedOutput.add(new StreamRecord(
       CRow(
-      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 1003))
+      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 5003))
     expectedOutput.add(new StreamRecord(
       CRow(
-      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 1003))
+      Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 5003))
 
     verify(expectedOutput, result, new RowResultSortComparator(6))
     testHarness.close()
@@ -510,10 +383,11 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new RowTimeBoundedRangeOver(
-        genAggFunction,
-        aggregationStateType,
-        cRT,
-        4000))
+        genMinMaxAggFunction,
+        minMaxAggregationStateType,
+        minMaxCRowType,
+        4000,
+        new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
 
     val testHarness =
       createHarnessTester(
@@ -573,6 +447,40 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(19000)
 
+    // test cleanup
+    testHarness.setProcessingTime(1000)
+    testHarness.processWatermark(20000)
+
+    // check that state is removed after max retention time
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 3000
+    testHarness.setProcessingTime(2500)
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 4500
+    testHarness.processWatermark(20010) // compute output
+
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(4499)
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(4500)
+    assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
+    // check that state is only removed if all data was processed
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong), true), 20011)) // clean-up 6500
+
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+
+    testHarness.processWatermark(20020) // schedule emission
+
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(8499) // clean-up
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(8500) // clean-up
+    assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -621,6 +529,16 @@ class OverWindowHarnessTest extends HarnessTestBase{
       CRow(
       Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001))
 
+    expectedOutput.add(new StreamRecord(
+      CRow(
+      Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true), 20011))
+
     verify(expectedOutput, result, new RowResultSortComparator(6))
     testHarness.close()
   }
@@ -630,10 +548,11 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new RowTimeBoundedRowsOver(
-        genAggFunction,
-        aggregationStateType,
-        cRT,
-        3))
+        genMinMaxAggFunction,
+        minMaxAggregationStateType,
+        minMaxCRowType,
+        3,
+        new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
 
     val testHarness =
       createHarnessTester(
@@ -689,6 +608,41 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(19000)
 
+    // test cleanup
+    testHarness.setProcessingTime(1000)
+    testHarness.processWatermark(20000)
+
+    // check that state is removed after max retention time
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 3000
+    testHarness.setProcessingTime(2500)
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 4500
+    testHarness.processWatermark(20010) // compute output
+
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(4499)
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(4500)
+    assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
+    // check that state is only removed if all data was processed
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong), true), 20011)) // clean-up 6500
+
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+
+    testHarness.processWatermark(20020) // schedule emission
+
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(8499) // clean-up
+    assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+    testHarness.setProcessingTime(8500) // clean-up
+    assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
+
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -736,6 +690,16 @@ class OverWindowHarnessTest extends HarnessTestBase{
       CRow(
       Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true), 12001))
 
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true), 20011))
+
     verify(expectedOutput, result, new RowResultSortComparator(6))
     testHarness.close()
   }
@@ -748,9 +712,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new RowTimeUnboundedRangeOver(
-        genAggFunction,
-        aggregationStateType,
-        cRT))
+        genMinMaxAggFunction,
+        minMaxAggregationStateType,
+        minMaxCRowType,
+        new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
 
     val testHarness =
       createHarnessTester(
@@ -760,6 +725,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.open()
 
+    testHarness.setProcessingTime(1000)
     testHarness.processWatermark(800)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801))
@@ -806,6 +772,30 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(19000)
 
+    // test cleanup
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(2999) // clean up timer is 3000, so nothing should happen
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(3000) // clean up is triggered
+    assert(testHarness.numKeyedStateEntries() == 0)
+
+    testHarness.processWatermark(20000)
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 5000
+    testHarness.setProcessingTime(2500)
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 5000
+
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000
+    testHarness.processWatermark(20010) // compute output
+
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(6999) // clean up timer is 3000, so nothing should happen
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(7000) // clean up is triggered
+    assert(testHarness.numKeyedStateEntries() == 0)
+
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -854,6 +844,13 @@ class OverWindowHarnessTest extends HarnessTestBase{
       CRow(
       Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001))
 
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+
     verify(expectedOutput, result, new RowResultSortComparator(6))
     testHarness.close()
   }
@@ -863,9 +860,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     val processFunction = new KeyedProcessOperator[String, CRow, CRow](
       new RowTimeUnboundedRowsOver(
-        genAggFunction,
-        aggregationStateType,
-        cRT))
+        genMinMaxAggFunction,
+        minMaxAggregationStateType,
+        minMaxCRowType,
+        new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
 
     val testHarness =
       createHarnessTester(
@@ -875,6 +873,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.open()
 
+    testHarness.setProcessingTime(1000)
     testHarness.processWatermark(800)
     testHarness.processElement(new StreamRecord(
       CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801))
@@ -921,6 +920,30 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(19000)
 
+    // test cleanup
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(2999) // clean up timer is 3000, so nothing should happen
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(3000) // clean up is triggered
+    assert(testHarness.numKeyedStateEntries() == 0)
+
+    testHarness.processWatermark(20000)
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 5000
+    testHarness.setProcessingTime(2500)
+    testHarness.processElement(new StreamRecord(
+      CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 5000
+
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000
+    testHarness.processWatermark(20010) // compute output
+
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(6999) // clean up timer is 3000, so nothing should happen
+    assert(testHarness.numKeyedStateEntries() > 0)
+    testHarness.setProcessingTime(7000) // clean up is triggered
+    assert(testHarness.numKeyedStateEntries() == 0)
+
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -968,6 +991,13 @@ class OverWindowHarnessTest extends HarnessTestBase{
       CRow(
       Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001))
 
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+    expectedOutput.add(new StreamRecord(
+      CRow(
+        Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+
     verify(expectedOutput, result, new RowResultSortComparator(6))
     testHarness.close()
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 3d79e22..c4e2433 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
@@ -28,7 +28,7 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) {
   override private[flink] def writeToSink[T](
       table: Table,
       sink: TableSink[T],
-      qConfig: QueryConfig): Unit = ???
+      queryConfig: QueryConfig): Unit = ???
 
   override protected def checkValidTableName(name: String): Unit = ???
 


[5/5] flink git commit: [FLINK-6483] [table] Add materialization of time indicators.

Posted by fh...@apache.org.
[FLINK-6483] [table] Add materialization of time indicators.

This closes #3862.


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

Branch: refs/heads/release-1.3
Commit: 609d5a32f77591beff39ea27dc48aedab9243c61
Parents: 6b61d15
Author: twalthr <tw...@apache.org>
Authored: Wed May 10 10:11:34 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri May 12 08:33:45 2017 +0200

----------------------------------------------------------------------
 .../table/api/StreamTableEnvironment.scala      |  39 +-
 .../calcite/RelTimeIndicatorConverter.scala     | 404 +++++++++++++------
 .../flink/table/codegen/CodeGenerator.scala     |  47 ++-
 .../flink/table/plan/nodes/CommonCalc.scala     |  13 +-
 .../table/plan/nodes/CommonCorrelate.scala      |  19 +-
 .../table/plan/nodes/dataset/DataSetCalc.scala  |   6 +-
 .../plan/nodes/dataset/DataSetCorrelate.scala   |   8 +-
 .../plan/nodes/datastream/DataStreamCalc.scala  |  14 +-
 .../nodes/datastream/DataStreamCorrelate.scala  |  23 +-
 .../datastream/StreamTableSourceScan.scala      |   4 +-
 .../plan/nodes/logical/FlinkLogicalCalc.scala   |   2 +-
 .../logical/FlinkLogicalTableSourceScan.scala   |   6 +-
 .../DataStreamLogicalWindowAggregateRule.scala  |  14 +-
 .../flink/table/plan/schema/RowSchema.scala     |  11 +-
 .../plan/schema/StreamTableSourceTable.scala    |   8 +-
 .../runtime/CRowCorrelateFlatMapRunner.scala    |  83 ----
 .../runtime/CRowCorrelateProcessRunner.scala    |  91 +++++
 .../flink/table/runtime/CRowFlatMapRunner.scala |  72 ----
 .../flink/table/runtime/CRowProcessRunner.scala |  80 ++++
 .../table/sources/DefinedTimeAttributes.scala   |  60 ---
 .../table/sources/definedTimeAttributes.scala   |  60 +++
 .../stream/StreamTableEnvironmentTest.scala     |  10 +-
 .../api/scala/stream/TableSourceTest.scala      |  22 +-
 .../calcite/RelTimeIndicatorConverterTest.scala | 351 ++++++++++++++++
 .../datastream/TimeAttributesITCase.scala       | 237 +++++++++++
 .../flink/table/utils/TableTestBase.scala       |   5 +
 26 files changed, 1267 insertions(+), 422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 d68da04..994ac80 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
@@ -18,38 +18,37 @@
 
 package org.apache.flink.table.api
 
-import _root_.java.util.concurrent.atomic.AtomicInteger
 import _root_.java.lang.{Boolean => JBool}
+import _root_.java.util.concurrent.atomic.AtomicInteger
 
 import org.apache.calcite.plan.RelOptUtil
 import org.apache.calcite.plan.hep.HepMatchOrder
-import org.apache.calcite.rel.{RelNode, RelVisitor}
 import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelNode, RelVisitor}
 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.TypeInformation
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
 import org.apache.flink.api.java.typeutils.TupleTypeInfo
 import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import org.apache.flink.streaming.api.TimeCharacteristic
 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, UpdateAsRetractionTrait}
-import org.apache.flink.table.plan.nodes.datastream._
+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, 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.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner}
 import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink}
-import org.apache.flink.table.sources.{StreamTableSource, TableSource}
+import org.apache.flink.table.sources.{DefinedRowtimeAttribute, StreamTableSource, TableSource}
 import org.apache.flink.table.typeutils.TypeCheckUtils
 import org.apache.flink.types.Row
 
@@ -111,6 +110,17 @@ abstract class StreamTableEnvironment(
   override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = {
     checkValidTableName(name)
 
+    // check if event-time is enabled
+    tableSource match {
+      case dra: DefinedRowtimeAttribute if
+          execEnv.getStreamTimeCharacteristic != TimeCharacteristic.EventTime =>
+
+        throw TableException(
+          s"A rowtime attribute requires an EventTime time characteristic in stream environment. " +
+            s"But is: ${execEnv.getStreamTimeCharacteristic}")
+      case _ => // ok
+    }
+
     tableSource match {
       case streamTableSource: StreamTableSource[_] =>
         registerTableInternal(name, new StreamTableSourceTable(streamTableSource))
@@ -390,6 +400,13 @@ abstract class StreamTableEnvironment(
     // validate and extract time attributes
     val (rowtime, proctime) = validateAndExtractTimeAttributes(streamType, fields)
 
+    // check if event-time is enabled
+    if (rowtime.isDefined && execEnv.getStreamTimeCharacteristic != TimeCharacteristic.EventTime) {
+      throw TableException(
+        s"A rowtime attribute requires an EventTime time characteristic in stream environment. " +
+          s"But is: ${execEnv.getStreamTimeCharacteristic}")
+    }
+
     val dataStreamTable = new DataStreamTable[T](
       dataStream,
       fieldIndexes,
@@ -518,9 +535,9 @@ abstract class StreamTableEnvironment(
     // 3. normalize the logical plan
     val normRuleSet = getNormRuleSet
     val normalizedPlan = if (normRuleSet.iterator().hasNext) {
-      runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, decorPlan, decorPlan.getTraitSet)
+      runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, convPlan, convPlan.getTraitSet)
     } else {
-      decorPlan
+      convPlan
     }
 
     // 4. optimize the logical Flink plan

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
index fa2e3ee..7ceb397 100644
--- 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
@@ -18,73 +18,43 @@
 
 package org.apache.flink.table.calcite
 
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl, RelRecordType, StructKind}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core._
 import org.apache.calcite.rel.logical._
-import org.apache.calcite.rel.{RelNode, RelShuttleImpl}
+import org.apache.calcite.rel.{RelNode, RelShuttle}
 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.api.{TableException, ValidationException}
 import org.apache.flink.table.functions.TimeMaterializationSqlFunction
 import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType
+import org.apache.flink.table.calcite.FlinkTypeFactory.isTimeIndicatorType
+import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate
 
 import scala.collection.JavaConversions._
+import scala.collection.mutable
 
 /**
   * 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 {
+class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
 
-  override def visit(project: LogicalProject): RelNode = {
-    // visit children and update inputs
-    val updatedProject = super.visit(project).asInstanceOf[LogicalProject]
+  private val timestamp = rexBuilder
+      .getTypeFactory
+      .asInstanceOf[FlinkTypeFactory]
+      .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
 
-    // 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(intersect: LogicalIntersect): RelNode =
+    throw new TableException("Logical intersect in a stream environment is not supported yet.")
 
   override def visit(union: LogicalUnion): RelNode = {
     // visit children and update inputs
-    val updatedUnion = super.visit(union).asInstanceOf[LogicalUnion]
+    val inputs = union.getInputs.map(_.accept(this))
 
     // make sure that time indicator types match
-    val inputTypes = updatedUnion.getInputs.map(_.getRowType)
+    val inputTypes = inputs.map(_.getRowType)
 
     val head = inputTypes.head.getFieldList.map(_.getType)
 
@@ -114,101 +84,269 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttleImpl {
         "Union fields with time attributes have different types.")
     }
 
-    updatedUnion
+    LogicalUnion.create(inputs, union.all)
   }
 
+  override def visit(aggregate: LogicalAggregate): RelNode = convertAggregate(aggregate)
+
+  override def visit(minus: LogicalMinus): RelNode =
+    throw new TableException("Logical minus in a stream environment is not supported yet.")
+
+  override def visit(sort: LogicalSort): RelNode =
+    throw new TableException("Logical sort in a stream environment is not supported yet.")
+
+  override def visit(`match`: LogicalMatch): RelNode =
+    throw new TableException("Logical match in a stream environment is not supported yet.")
+
   override def visit(other: RelNode): RelNode = other match {
-    case scan: LogicalTableFunctionScan if
-        stack.size() > 0 && stack.peek().isInstanceOf[LogicalCorrelate] =>
+
+    case uncollect: Uncollect =>
       // 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
-      )
+      val input = uncollect.getInput.accept(this)
+      Uncollect.create(uncollect.getTraitSet, input, uncollect.withOrdinality)
+
+    case scan: LogicalTableFunctionScan =>
+      scan
+
+    case aggregate: LogicalWindowAggregate =>
+      val convAggregate = convertAggregate(aggregate)
+
+      LogicalWindowAggregate.create(
+        aggregate.getWindow,
+        aggregate.getNamedProperties,
+        convAggregate)
 
     case _ =>
-      super.visit(other)
+      throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}")
   }
 
-  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)
+
+  override def visit(exchange: LogicalExchange): RelNode =
+    throw new TableException("Logical exchange in a stream environment is not supported yet.")
+
+  override def visit(scan: TableScan): RelNode = scan
+
+  override def visit(scan: TableFunctionScan): RelNode =
+    throw new TableException("Table function scan in a stream environment is not supported yet.")
+
+  override def visit(values: LogicalValues): RelNode = values
+
+  override def visit(filter: LogicalFilter): RelNode = {
+    // visit children and update inputs
+    val input = filter.getInput.accept(this)
+
+    // 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,
+      input.getRowType.getFieldList.map(_.getType))
+
+    val condition = filter.getCondition.accept(materializer)
+    LogicalFilter.create(input, condition)
   }
-}
 
-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 visit(project: LogicalProject): RelNode = {
+    // visit children and update inputs
+    val input = project.getInput.accept(this)
+
+    // 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,
+      input.getRowType.getFieldList.map(_.getType))
+
+    val projects = project.getProjects.map(_.accept(materializer))
+    val fieldNames = project.getRowType.getFieldNames
+    LogicalProject.create(input, projects, fieldNames)
+  }
+
+  override def visit(join: LogicalJoin): RelNode =
+    throw new TableException("Logical join in a stream environment is not supported yet.")
+
+  override def visit(correlate: LogicalCorrelate): RelNode = {
+    // visit children and update inputs
+    val inputs = correlate.getInputs.map(_.accept(this))
+
+    val right = inputs(1) match {
+      case scan: LogicalTableFunctionScan =>
+        // visit children and update inputs
+        val scanInputs = scan.getInputs.map(_.accept(this))
+
+        // 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,
+          inputs.head.getRowType.getFieldList.map(_.getType))
+
+        val call = scan.getCall.accept(materializer)
+        LogicalTableFunctionScan.create(
+          scan.getCluster,
+          scanInputs,
+          call,
+          scan.getElementType,
+          scan.getRowType,
+          scan.getColumnMappings)
+
+      case _ =>
+        inputs(1)
     }
+
+    LogicalCorrelate.create(
+      inputs.head,
+      right,
+      correlate.getCorrelationId,
+      correlate.getRequiredColumns,
+      correlate.getJoinType)
   }
 
-  override def visitCall(call: RexCall): RexNode = {
-    val updatedCall = super.visitCall(call).asInstanceOf[RexCall]
+  private def convertAggregate(aggregate: Aggregate): LogicalAggregate = {
+    // visit children and update inputs
+    val input = aggregate.getInput.accept(this)
+
+    // add a project to materialize aggregation arguments/grouping keys
+
+    val refIndices = mutable.Set[Int]()
+
+    // check arguments of agg calls
+    aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) {
+        // count(*) has an empty argument list
+        (0 until input.getRowType.getFieldCount).foreach(refIndices.add)
+      } else {
+        // for other aggregations
+        call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add)
+      })
 
-    // skip materialization for special operators
-    updatedCall.getOperator match {
-      case SqlStdOperatorTable.SESSION | SqlStdOperatorTable.HOP | SqlStdOperatorTable.TUMBLE =>
-        return updatedCall
+    // check grouping sets
+    aggregate.getGroupSets.foreach(set =>
+      set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add)
+    )
 
-      case _ => // do nothing
+    val needsMaterialization = refIndices.exists(idx =>
+      isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType))
+
+    // create project if necessary
+    val projectedInput = if (needsMaterialization) {
+
+      // insert or merge with input project if
+      // a time attribute is accessed and needs to be materialized
+      input match {
+
+        // merge
+        case lp: LogicalProject =>
+          val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) =>
+            if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) {
+              rexBuilder.makeCall(
+                TimeMaterializationSqlFunction,
+                expr)
+            } else {
+              expr
+            }
+          }
+
+          LogicalProject.create(
+            lp.getInput,
+            projects,
+            input.getRowType.getFieldNames)
+
+        // new project
+        case _ =>
+          val projects = input.getRowType.getFieldList.map { field =>
+            if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) {
+              rexBuilder.makeCall(
+                TimeMaterializationSqlFunction,
+                new RexInputRef(field.getIndex, field.getType))
+            } else {
+              new RexInputRef(field.getIndex, field.getType)
+            }
+          }
+
+          LogicalProject.create(
+            input,
+            projects,
+            input.getRowType.getFieldNames)
+      }
+    } else {
+      // no project necessary
+      input
     }
 
-    // materialize operands with time indicators
-    val materializedOperands = updatedCall.getOperands.map { o =>
-      if (isTimeIndicatorType(o.getType)) {
-        rexBuilder.makeCall(TimeMaterializationSqlFunction, o)
+    // remove time indicator type as agg call return type
+    val updatedAggCalls = aggregate.getAggCallList.map { call =>
+      val callType = if (isTimeIndicatorType(call.getType)) {
+        timestamp
       } else {
-        o
+        call.getType
       }
+      AggregateCall.create(
+        call.getAggregation,
+        call.isDistinct,
+        call.getArgList,
+        call.filterArg,
+        callType,
+        call.name)
     }
 
-    // remove time indicator return type
-    if (isTimeIndicatorType(updatedCall.getType)) {
-      updatedCall.clone(timestamp, materializedOperands)
-    } else {
-      updatedCall.clone(updatedCall.getType, materializedOperands)
+    LogicalAggregate.create(
+      projectedInput,
+      aggregate.indicator,
+      aggregate.getGroupSet,
+      aggregate.getGroupSets,
+      updatedAggCalls)
+  }
+
+  class RexTimeIndicatorMaterializer(
+      private val rexBuilder: RexBuilder,
+      private val input: Seq[RelDataType])
+    extends RexShuttle {
+
+    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]
+
+      // materialize operands with time indicators
+      val materializedOperands = updatedCall.getOperator match {
+
+        // skip materialization for special operators
+        case SqlStdOperatorTable.SESSION | SqlStdOperatorTable.HOP | SqlStdOperatorTable.TUMBLE =>
+          updatedCall.getOperands.toList
+
+        case _ =>
+          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)
+      }
     }
   }
 }
@@ -217,6 +355,30 @@ object RelTimeIndicatorConverter {
 
   def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = {
     val converter = new RelTimeIndicatorConverter(rexBuilder)
-    rootRel.accept(converter)
+    val convertedRoot = rootRel.accept(converter)
+
+    var needsConversion = false
+
+    // materialize all remaining time indicators
+    val projects = convertedRoot.getRowType.getFieldList.map(field =>
+      if (isTimeIndicatorType(field.getType)) {
+        needsConversion = true
+        rexBuilder.makeCall(
+          TimeMaterializationSqlFunction,
+          new RexInputRef(field.getIndex, field.getType))
+      } else {
+        new RexInputRef(field.getIndex, field.getType)
+      }
+    )
+
+    // add final conversion
+    if (needsConversion) {
+      LogicalProject.create(
+      convertedRoot,
+      projects,
+      convertedRoot.getRowType.getFieldNames)
+    } else {
+      convertedRoot
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 25addbc..036889f 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
@@ -34,6 +34,7 @@ import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.java.typeutils._
 import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenUtils._
@@ -238,6 +239,11 @@ class CodeGenerator(
   var outRecordTerm = "out"
 
   /**
+    * @return term of the [[ProcessFunction]]'s context
+    */
+  var contextTerm = "ctx"
+
+  /**
     * @return returns if null checking is enabled
     */
   def nullCheck: Boolean = config.getNullCheck
@@ -699,6 +705,17 @@ class CodeGenerator(
           List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;",
                s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;"))
       }
+
+      // ProcessFunction
+      else if (clazz == classOf[ProcessFunction[_, _]]) {
+        val baseClass = classOf[ProcessFunction[_, _]]
+        val inputTypeTerm = boxedTypeTermForTypeInfo(input1)
+        (baseClass,
+          s"void processElement(Object _in1, " +
+            s"org.apache.flink.streaming.api.functions.ProcessFunction.Context $contextTerm," +
+            s"org.apache.flink.util.Collector $collectorTerm)",
+          List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"))
+      }
       else {
         // TODO more functions
         throw new CodeGenException("Unsupported Function.")
@@ -1312,9 +1329,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
+    // special case: time materialization
     if (call.getOperator == TimeMaterializationSqlFunction) {
-      throw new CodeGenException("Access to time attributes is not possible yet.")
+      return generateRecordTimestamp(
+        FlinkTypeFactory.isRowtimeIndicatorType(call.getOperands.get(0).getType)
+      )
     }
 
     val operands = call.getOperands.map(_.accept(this))
@@ -1840,6 +1859,30 @@ class CodeGenerator(
     }
   }
 
+  private[flink] def generateRecordTimestamp(isEventTime: Boolean): GeneratedExpression = {
+    val resultTerm = newName("result")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+
+    val resultCode = if (isEventTime) {
+      s"""
+        |$resultTypeTerm $resultTerm;
+        |if ($contextTerm.timestamp() == null) {
+        |  throw new RuntimeException("Rowtime timestamp is null. Please make sure that a proper " +
+        |    "TimestampAssigner is defined and the stream environment uses the EventTime time " +
+        |    "characteristic.");
+        |}
+        |else {
+        |  $resultTerm = $contextTerm.timestamp();
+        |}
+        |""".stripMargin
+    } else {
+      s"""
+        |$resultTypeTerm $resultTerm = $contextTerm.timerService().currentProcessingTime();
+        |""".stripMargin
+    }
+    GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP)
+  }
+
   // ----------------------------------------------------------------------------------------------
   // Reusable code snippets
   // ----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 e875587..9b486e4 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,7 +20,7 @@ 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
+import org.apache.flink.api.common.functions.Function
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.{CodeGenerator, GeneratedFunction}
@@ -30,16 +30,17 @@ import org.apache.flink.types.Row
 import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
 
-trait CommonCalc[T] {
+trait CommonCalc {
 
-  private[flink] def generateFunction(
+  private[flink] def generateFunction[T <: Function](
       generator: CodeGenerator,
       ruleDescription: String,
       inputSchema: RowSchema,
       returnSchema: RowSchema,
       calcProgram: RexProgram,
-      config: TableConfig):
-    GeneratedFunction[FlatMapFunction[Row, Row], Row] = {
+      config: TableConfig,
+      functionClass: Class[T]):
+    GeneratedFunction[T, Row] = {
 
     val expandedExpressions = calcProgram
       .getProjectList
@@ -92,7 +93,7 @@ trait CommonCalc[T] {
 
     generator.generateFunction(
       ruleDescription,
-      classOf[FlatMapFunction[Row, Row]],
+      functionClass,
       body,
       returnSchema.physicalTypeInfo)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 c95f2f7..874bea2 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
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexShuttle}
 import org.apache.calcite.sql.SemiJoinType
-import org.apache.flink.api.common.functions.FlatMapFunction
+import org.apache.flink.api.common.functions.{FlatMapFunction, Function}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.codegen.CodeGenUtils.primitiveDefaultValue
@@ -36,22 +36,22 @@ import scala.collection.JavaConverters._
 /**
   * Join a user-defined table function
   */
-trait CommonCorrelate[T] {
+trait CommonCorrelate {
 
   /**
     * Generates the flat map function to run the user-defined table function.
     */
-  private[flink] def generateFunction(
+  private[flink] def generateFunction[T <: Function](
     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] = {
+    ruleDescription: String,
+    functionClass: Class[T]):
+  GeneratedFunction[T, Row] = {
 
     val functionGenerator = new CodeGenerator(
       config,
@@ -89,7 +89,7 @@ trait CommonCorrelate[T] {
       val outerResultExpr = functionGenerator.generateResultExpression(
         input1AccessExprs ++ input2NullExprs,
         returnSchema.physicalTypeInfo,
-        rowType.getFieldNames.asScala)
+        returnSchema.physicalFieldNames)
       body +=
         s"""
            |boolean hasOutput = $collectorTerm.isCollected();
@@ -104,7 +104,7 @@ trait CommonCorrelate[T] {
 
     functionGenerator.generateFunction(
       ruleDescription,
-      classOf[FlatMapFunction[Row, Row]],
+      functionClass,
       body,
       returnSchema.physicalTypeInfo)
   }
@@ -117,7 +117,6 @@ trait CommonCorrelate[T] {
     inputSchema: RowSchema,
     udtfTypeInfo: TypeInformation[Any],
     returnSchema: RowSchema,
-    rowType: RelDataType,
     condition: Option[RexNode],
     pojoFieldMapping: Option[Array[Int]])
   : GeneratedCollector = {
@@ -135,7 +134,7 @@ trait CommonCorrelate[T] {
     val crossResultExpr = generator.generateResultExpression(
       input1AccessExprs ++ input2AccessExprs,
       returnSchema.physicalTypeInfo,
-      rowType.getFieldNames.asScala)
+      returnSchema.physicalFieldNames)
 
     val collectorCode = if (condition.isEmpty) {
       s"""

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 e340a8c..9a9f738 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,6 +24,7 @@ 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
@@ -46,7 +47,7 @@ class DataSetCalc(
     calcProgram: RexProgram,
     ruleDescription: String)
   extends Calc(cluster, traitSet, input, calcProgram)
-  with CommonCalc[Row]
+  with CommonCalc
   with DataSetRel {
 
   override def deriveRowType(): RelDataType = rowRelDataType
@@ -95,7 +96,8 @@ class DataSetCalc(
       new RowSchema(getInput.getRowType),
       new RowSchema(getRowType),
       calcProgram,
-      config)
+      config,
+      classOf[FlatMapFunction[Row, Row]])
 
     val runner = new FlatMapRunner(genFunction.name, genFunction.code, returnType)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 49ead26..731d2e5 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
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
 import org.apache.calcite.rex.{RexCall, RexNode}
 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.api.java.DataSet
 import org.apache.flink.api.java.typeutils.RowTypeInfo
@@ -49,7 +50,7 @@ class DataSetCorrelate(
     joinType: SemiJoinType,
     ruleDescription: String)
   extends SingleRel(cluster, traitSet, inputNode)
-  with CommonCorrelate[Row]
+  with CommonCorrelate
   with DataSetRel {
 
   override def deriveRowType() = relRowType
@@ -109,18 +110,17 @@ class DataSetCorrelate(
       new RowSchema(getInput.getRowType),
       udtfTypeInfo,
       new RowSchema(getRowType),
-      rowType,
       joinType,
       rexCall,
       pojoFieldMapping,
-      ruleDescription)
+      ruleDescription,
+      classOf[FlatMapFunction[Row, Row]])
 
     val collector = generateCollector(
       config,
       new RowSchema(getInput.getRowType),
       udtfTypeInfo,
       new RowSchema(getRowType),
-      rowType,
       condition,
       pojoFieldMapping)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 5f270f6..f75efc8 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
@@ -25,11 +25,12 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rel.{RelNode, RelWriter}
 import org.apache.calcite.rex.RexProgram
 import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.table.api.{StreamQueryConfig, 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.runtime.CRowFlatMapRunner
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.CRowProcessRunner
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 /**
@@ -45,7 +46,7 @@ class DataStreamCalc(
     calcProgram: RexProgram,
     ruleDescription: String)
   extends Calc(cluster, traitSet, input, calcProgram)
-  with CommonCalc[CRow]
+  with CommonCalc
   with DataStreamRel {
 
   override def deriveRowType(): RelDataType = schema.logicalType
@@ -101,17 +102,18 @@ class DataStreamCalc(
       inputSchema,
       schema,
       calcProgram,
-      config)
+      config,
+      classOf[ProcessFunction[CRow, CRow]])
 
     val inputParallelism = inputDataStream.getParallelism
 
-    val mapFunc = new CRowFlatMapRunner(
+    val processFunc = new CRowProcessRunner(
       genFunction.name,
       genFunction.code,
       CRowTypeInfo(schema.physicalTypeInfo))
 
     inputDataStream
-      .flatMap(mapFunc)
+      .process(processFunc)
       .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/609d5a32/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 5b32b10..b7165cd 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
@@ -23,12 +23,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.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.table.api.{StreamQueryConfig, 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.table.runtime.CRowCorrelateFlatMapRunner
+import org.apache.flink.table.runtime.CRowCorrelateProcessRunner
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 /**
@@ -46,7 +47,7 @@ class DataStreamCorrelate(
     joinType: SemiJoinType,
     ruleDescription: String)
   extends SingleRel(cluster, traitSet, input)
-  with CommonCorrelate[CRow]
+  with CommonCorrelate
   with DataStreamRel {
 
   override def deriveRowType() = schema.logicalType
@@ -90,7 +91,6 @@ class DataStreamCorrelate(
 
     // we do not need to specify input type
     val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
-    val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo]
 
     val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]
     val rexCall = funcRel.getCall.asInstanceOf[RexCall]
@@ -98,37 +98,36 @@ class DataStreamCorrelate(
     val pojoFieldMapping = Some(sqlFunction.getPojoFieldMapping)
     val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
 
-    val flatMap = generateFunction(
+    val process = generateFunction(
       config,
       inputSchema,
       udtfTypeInfo,
       schema,
-      getRowType,
       joinType,
       rexCall,
       pojoFieldMapping,
-      ruleDescription)
+      ruleDescription,
+      classOf[ProcessFunction[CRow, CRow]])
 
     val collector = generateCollector(
       config,
       inputSchema,
       udtfTypeInfo,
       schema,
-      getRowType,
       condition,
       pojoFieldMapping)
 
-    val mapFunc = new CRowCorrelateFlatMapRunner(
-      flatMap.name,
-      flatMap.code,
+    val processFunc = new CRowCorrelateProcessRunner(
+      process.name,
+      process.code,
       collector.name,
       collector.code,
-      CRowTypeInfo(flatMap.returnType))
+      CRowTypeInfo(process.returnType))
 
     val inputParallelism = inputDS.getParallelism
 
     inputDS
-      .flatMap(mapFunc)
+      .process(processFunc)
       // 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/609d5a32/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 51e609f..72ecac5 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
@@ -49,7 +49,7 @@ class StreamTableSourceScan(
     val fieldCnt = fieldNames.length
 
     val rowtime = tableSource match {
-      case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+      case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null =>
         val rowtimeAttribute = timeSource.getRowtimeAttribute
         Some((fieldCnt, rowtimeAttribute))
       case _ =>
@@ -57,7 +57,7 @@ class StreamTableSourceScan(
     }
 
     val proctime = tableSource match {
-      case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+      case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null =>
         val proctimeAttribute = timeSource.getProctimeAttribute
         Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
       case _ =>

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 0ca079e..ec90392 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[Any] {
+  with CommonCalc {
 
   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/609d5a32/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 a2777ec..3ae949e 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.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, TableSource}
+import org.apache.flink.table.sources.{DefinedProctimeAttribute, DefinedRowtimeAttribute, TableSource}
 
 import scala.collection.JavaConverters._
 
@@ -54,7 +54,7 @@ class FlinkLogicalTableSourceScan(
     val fieldCnt = fieldNames.length
 
     val rowtime = tableSource match {
-      case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+      case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null =>
         val rowtimeAttribute = timeSource.getRowtimeAttribute
         Some((fieldCnt, rowtimeAttribute))
       case _ =>
@@ -62,7 +62,7 @@ class FlinkLogicalTableSourceScan(
     }
 
     val proctime = tableSource match {
-      case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+      case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null =>
         val proctimeAttribute = timeSource.getProctimeAttribute
         Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
       case _ =>

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 28efcf5..d57d4cc 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
@@ -26,7 +26,7 @@ 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.calcite.FlinkTypeFactory
-import org.apache.flink.table.expressions.{Literal, UnresolvedFieldReference}
+import org.apache.flink.table.expressions.{Literal, ResolvedFieldReference, UnresolvedFieldReference}
 import org.apache.flink.table.plan.rules.common.LogicalWindowAggregateRule
 import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
 
@@ -68,10 +68,12 @@ class DataStreamLogicalWindowAggregateRule
         case _ => throw new TableException("Only constant window descriptors are supported.")
       }
 
-    def getOperandAsTimeIndicator(call: RexCall, idx: Int): String =
+    def getOperandAsTimeIndicator(call: RexCall, idx: Int): ResolvedFieldReference =
       call.getOperands.get(idx) match {
         case v: RexInputRef if FlinkTypeFactory.isTimeIndicatorType(v.getType) =>
-          rowType.getFieldList.get(v.getIndex).getName
+          ResolvedFieldReference(
+            rowType.getFieldList.get(v.getIndex).getName,
+            FlinkTypeFactory.toTypeInfo(v.getType))
         case _ =>
           throw new TableException("Window can only be defined over a time attribute column.")
       }
@@ -82,7 +84,7 @@ class DataStreamLogicalWindowAggregateRule
         val interval = getOperandAsLong(windowExpr, 1)
         val w = Tumble.over(Literal(interval, TimeIntervalTypeInfo.INTERVAL_MILLIS))
 
-        w.on(UnresolvedFieldReference(time)).as("w$")
+        w.on(time).as("w$")
 
       case SqlStdOperatorTable.HOP =>
         val time = getOperandAsTimeIndicator(windowExpr, 0)
@@ -91,14 +93,14 @@ class DataStreamLogicalWindowAggregateRule
           .over(Literal(size, TimeIntervalTypeInfo.INTERVAL_MILLIS))
           .every(Literal(slide, TimeIntervalTypeInfo.INTERVAL_MILLIS))
 
-        w.on(UnresolvedFieldReference(time)).as("w$")
+        w.on(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))
 
-        w.on(UnresolvedFieldReference(time)).as("w$")
+        w.on(time).as("w$")
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
index b42be82..ccbe44d 100644
--- 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
@@ -20,11 +20,12 @@ 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.calcite.rex.{RexCall, 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.table.functions.TimeMaterializationSqlFunction
 import org.apache.flink.types.Row
 
 import scala.collection.JavaConversions._
@@ -76,6 +77,14 @@ class RowSchema(private val logicalRowType: RelDataType) {
     override def visitInputRef(inputRef: RexInputRef): RexNode = {
       new RexInputRef(mapIndex(inputRef.getIndex), inputRef.getType)
     }
+
+    override def visitCall(call: RexCall): RexNode = call.getOperator match {
+      // we leave time indicators unchanged yet
+      // the index becomes invalid but right now we are only
+      // interested in the type of the input reference
+      case TimeMaterializationSqlFunction => call
+      case _ => super.visitCall(call)
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
index 75deca5..fa15288 100644
--- 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
@@ -19,10 +19,10 @@
 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.api.{TableEnvironment, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.stats.FlinkStatistic
-import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, TableSource}
+import org.apache.flink.table.sources.{DefinedProctimeAttribute, DefinedRowtimeAttribute, TableSource}
 
 class StreamTableSourceTable[T](
     override val tableSource: TableSource[T],
@@ -39,7 +39,7 @@ class StreamTableSourceTable[T](
     val fieldCnt = fieldNames.length
 
     val rowtime = tableSource match {
-      case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+      case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null =>
         val rowtimeAttribute = timeSource.getRowtimeAttribute
         Some((fieldCnt, rowtimeAttribute))
       case _ =>
@@ -47,7 +47,7 @@ class StreamTableSourceTable[T](
     }
 
     val proctime = tableSource match {
-      case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+      case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null =>
         val proctimeAttribute = timeSource.getProctimeAttribute
         Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
       case _ =>

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
deleted file mode 100644
index ff3821a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
+++ /dev/null
@@ -1,83 +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.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 var 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/609d5a32/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala
new file mode 100644
index 0000000..4f0a785
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala
@@ -0,0 +1,91 @@
+/*
+ * 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.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+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 CorrelateProcessRunner with [[CRow]] input and [[CRow]] output.
+  */
+class CRowCorrelateProcessRunner(
+    processName: String,
+    processCode: String,
+    collectorName: String,
+    collectorCode: String,
+    @transient var returnType: TypeInformation[CRow])
+  extends ProcessFunction[CRow, CRow]
+  with ResultTypeQueryable[CRow]
+  with Compiler[Any] {
+
+  val LOG: Logger = LoggerFactory.getLogger(this.getClass)
+
+  private var function: ProcessFunction[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 ProcessFunction: $processName \n\n Code:\n$processCode")
+    val processClazz = compile(getRuntimeContext.getUserCodeClassLoader, processName, processCode)
+    val constructor = processClazz.getConstructor(classOf[TableFunctionCollector[_]])
+    LOG.debug("Instantiating ProcessFunction.")
+    function = constructor.newInstance(collector).asInstanceOf[ProcessFunction[Row, Row]]
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
+  }
+
+  override def processElement(
+      in: CRow,
+      ctx: ProcessFunction[CRow, CRow]#Context,
+      out: Collector[CRow])
+    : Unit = {
+
+    cRowWrapper.out = out
+    cRowWrapper.setChange(in.change)
+
+    collector.setCollector(cRowWrapper)
+    collector.setInput(in.row)
+    collector.reset()
+
+    function.processElement(
+      in.row,
+      ctx.asInstanceOf[ProcessFunction[Row, Row]#Context],
+      cRowWrapper)
+  }
+
+  override def getProducedType: TypeInformation[CRow] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
deleted file mode 100644
index 9701cb9..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
+++ /dev/null
@@ -1,72 +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.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 var 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)
-  }
-}
-
-

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

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
deleted file mode 100644
index 6d87663..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
+++ /dev/null
@@ -1,60 +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.sources
-
-/**
-  * 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 DefinedRowTimeAttribute {
-
-  /**
-    * 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: 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 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: String
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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..d381115
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.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.sources
+
+/**
+  * 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 DefinedRowtimeAttribute {
+
+  /**
+    * 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: 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 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: String
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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
index e9384c7..7797f22 100644
--- 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
@@ -19,18 +19,16 @@
 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.TimeCharacteristic
 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.api.{TableEnvironment, TableException, Types}
 import org.apache.flink.table.utils.TableTestBase
 import org.junit.Test
 import org.mockito.Mockito.{mock, when}
@@ -151,7 +149,9 @@ class StreamTableEnvironmentTest extends TableTestBase {
   private def prepareSchemaExpressionParser:
     (JStreamTableEnv, DataStream[JTuple5[JLong, JInt, String, JInt, JLong]]) = {
 
-    val jTEnv = TableEnvironment.getTableEnvironment(mock(classOf[JStreamExecEnv]))
+    val jStreamExecEnv = mock(classOf[JStreamExecEnv])
+    when(jStreamExecEnv.getStreamTimeCharacteristic).thenReturn(TimeCharacteristic.EventTime)
+    val jTEnv = TableEnvironment.getTableEnvironment(jStreamExecEnv)
 
     val sType = new TupleTypeInfo(Types.LONG, Types.INT, Types.STRING, Types.INT, Types.LONG)
       .asInstanceOf[TupleTypeInfo[JTuple5[JLong, JInt, String, JInt, JLong]]]

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 18066c9..cda90f7 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
@@ -24,7 +24,7 @@ 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.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
@@ -35,7 +35,7 @@ class TableSourceTest extends TableTestBase {
   @Test
   def testRowTimeTableSourceSimple(): Unit = {
     val util = streamTestUtil()
-    util.tEnv.registerTableSource("rowTimeT", new TestRowTimeSource("addTime"))
+    util.tEnv.registerTableSource("rowTimeT", new TestRowtimeSource("addTime"))
 
     val t = util.tEnv.scan("rowTimeT").select("addTime, id, name, val")
 
@@ -43,7 +43,7 @@ class TableSourceTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         "StreamTableSourceScan(table=[[rowTimeT]], fields=[id, val, name, addTime])",
-        term("select", "addTime", "id", "name", "val")
+        term("select", "TIME_MATERIALIZATION(addTime) AS addTime", "id", "name", "val")
       )
     util.verifyTable(t, expected)
   }
@@ -51,7 +51,7 @@ class TableSourceTest extends TableTestBase {
   @Test
   def testRowTimeTableSourceGroupWindow(): Unit = {
     val util = streamTestUtil()
-    util.tEnv.registerTableSource("rowTimeT", new TestRowTimeSource("addTime"))
+    util.tEnv.registerTableSource("rowTimeT", new TestRowtimeSource("addTime"))
 
     val t = util.tEnv.scan("rowTimeT")
       .filter("val > 100")
@@ -82,7 +82,7 @@ class TableSourceTest extends TableTestBase {
   @Test
   def testProcTimeTableSourceSimple(): Unit = {
     val util = streamTestUtil()
-    util.tEnv.registerTableSource("procTimeT", new TestProcTimeSource("pTime"))
+    util.tEnv.registerTableSource("procTimeT", new TestProctimeSource("pTime"))
 
     val t = util.tEnv.scan("procTimeT").select("pTime, id, name, val")
 
@@ -90,7 +90,7 @@ class TableSourceTest extends TableTestBase {
       unaryNode(
         "DataStreamCalc",
         "StreamTableSourceScan(table=[[procTimeT]], fields=[id, val, name, pTime])",
-        term("select", "pTime", "id", "name", "val")
+        term("select", "TIME_MATERIALIZATION(pTime) AS pTime", "id", "name", "val")
       )
     util.verifyTable(t, expected)
   }
@@ -98,7 +98,7 @@ class TableSourceTest extends TableTestBase {
   @Test
   def testProcTimeTableSourceOverWindow(): Unit = {
     val util = streamTestUtil()
-    util.tEnv.registerTableSource("procTimeT", new TestProcTimeSource("pTime"))
+    util.tEnv.registerTableSource("procTimeT", new TestProctimeSource("pTime"))
 
     val t = util.tEnv.scan("procTimeT")
       .window(Over partitionBy 'id orderBy 'pTime preceding 2.hours as 'w)
@@ -123,8 +123,8 @@ class TableSourceTest extends TableTestBase {
   }
 }
 
-class TestRowTimeSource(timeField: String)
-    extends StreamTableSource[Row] with DefinedRowTimeAttribute {
+class TestRowtimeSource(timeField: String)
+    extends StreamTableSource[Row] with DefinedRowtimeAttribute {
 
   override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = ???
 
@@ -137,8 +137,8 @@ class TestRowTimeSource(timeField: String)
   }
 }
 
-class TestProcTimeSource(timeField: String)
-    extends StreamTableSource[Row] with DefinedProcTimeAttribute {
+class TestProctimeSource(timeField: String)
+    extends StreamTableSource[Row] with DefinedProctimeAttribute {
 
   override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = ???
 


[2/5] flink git commit: [FLINK-6491] [table] Add QueryConfig and state clean up for over-windowed aggregates.

Posted by fh...@apache.org.
[FLINK-6491] [table] Add QueryConfig and state clean up for over-windowed aggregates.

This closes #3863.


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

Branch: refs/heads/release-1.3
Commit: 6b61d1539386388c1ff38183bb95e86ae70cc0f0
Parents: 003f81a
Author: sunjincheng121 <su...@gmail.com>
Authored: Tue May 9 14:36:42 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri May 12 08:33:44 2017 +0200

----------------------------------------------------------------------
 .../flink/table/api/BatchTableEnvironment.scala |   8 +-
 .../apache/flink/table/api/QueryConfig.scala    | 102 -----
 .../table/api/StreamTableEnvironment.scala      |  38 +-
 .../table/api/java/StreamTableEnvironment.scala |  35 +-
 .../apache/flink/table/api/queryConfig.scala    | 102 +++++
 .../api/scala/StreamTableEnvironment.scala      |  20 +-
 .../table/api/scala/TableConversions.scala      |  13 +-
 .../org/apache/flink/table/api/table.scala      |   6 +-
 .../plan/nodes/datastream/DataStreamCalc.scala  |   5 +-
 .../nodes/datastream/DataStreamCorrelate.scala  |   4 +-
 .../datastream/DataStreamGroupAggregate.scala   |  10 +-
 .../DataStreamGroupWindowAggregate.scala        |   4 +-
 .../datastream/DataStreamOverAggregate.scala    |  34 +-
 .../plan/nodes/datastream/DataStreamRel.scala   |   4 +-
 .../plan/nodes/datastream/DataStreamScan.scala  |   2 +-
 .../plan/nodes/datastream/DataStreamUnion.scala |   6 +-
 .../nodes/datastream/DataStreamValues.scala     |   2 +-
 .../datastream/StreamTableSourceScan.scala      |   2 +-
 .../table/runtime/aggregate/AggregateUtil.scala |  32 +-
 .../aggregate/GroupAggProcessFunction.scala     |  44 +-
 .../aggregate/ProcTimeBoundedRangeOver.scala    |  26 +-
 .../aggregate/ProcTimeBoundedRowsOver.scala     |  20 +-
 .../ProcTimeUnboundedNonPartitionedOver.scala   |  20 +-
 .../ProcTimeUnboundedPartitionedOver.scala      |  20 +-
 .../ProcessFunctionWithCleanupState.scala       |  85 ++++
 .../aggregate/RowTimeBoundedRangeOver.scala     |  44 +-
 .../aggregate/RowTimeBoundedRowsOver.scala      |  41 +-
 .../aggregate/RowTimeUnboundedOver.scala        |  44 +-
 .../stream/table/GroupAggregationsITCase.scala  |  13 +-
 ...ProcessingOverRangeProcessFunctionTest.scala | 336 --------------
 .../table/runtime/harness/HarnessTestBase.scala | 281 +++++++++++-
 .../runtime/harness/NonWindowHarnessTest.scala  | 157 +++++++
 .../runtime/harness/OverWindowHarnessTest.scala | 458 ++++++++++---------
 .../table/utils/MockTableEnvironment.scala      |   2 +-
 34 files changed, 1209 insertions(+), 811 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 f33c187..3c0f51b 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
@@ -113,17 +113,17 @@ abstract class BatchTableEnvironment(
     *
     * @param table The [[Table]] to write.
     * @param sink The [[TableSink]] to write the [[Table]] to.
-    * @param qConfig The configuration for the query to generate.
+    * @param queryConfig The configuration for the query to generate.
     * @tparam T The expected type of the [[DataSet]] which represents the [[Table]].
     */
   override private[flink] def writeToSink[T](
       table: Table,
       sink: TableSink[T],
-      qConfig: QueryConfig): Unit = {
+      queryConfig: QueryConfig): Unit = {
 
     // We do not pass the configuration on, because there is nothing to configure for batch queries.
-    val bQConfig = qConfig match {
-      case batchConfig: BatchQueryConfig => batchConfig
+    queryConfig match {
+      case _: BatchQueryConfig =>
       case _ =>
         throw new TableException("BatchQueryConfig required to configure batch query.")
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
deleted file mode 100644
index 8e8b5ac..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
+++ /dev/null
@@ -1,102 +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
-
-import _root_.java.io.Serializable
-import org.apache.flink.api.common.time.Time
-
-class QueryConfig private[table] extends Serializable {}
-
-/**
-  * The [[BatchQueryConfig]] holds parameters to configure the behavior of batch queries.
-  */
-class BatchQueryConfig private[table] extends QueryConfig
-
-/**
-  * The [[StreamQueryConfig]] holds parameters to configure the behavior of streaming queries.
-  *
-  * An empty [[StreamQueryConfig]] can be generated using the [[StreamTableEnvironment.qConf]]
-  * method.
-  */
-class StreamQueryConfig private[table] extends QueryConfig {
-
-  /**
-    * The minimum time until state which was not updated will be retained.
-    * State might be cleared and removed if it was not updated for the defined period of time.
-    */
-  private var minIdleStateRetentionTime: Long = Long.MinValue
-
-  /**
-    * The maximum time until state which was not updated will be retained.
-    * State will be cleared and removed if it was not updated for the defined period of time.
-    */
-  private var maxIdleStateRetentionTime: Long = Long.MinValue
-
-  /**
-    * Specifies the time interval for how long idle state, i.e., state which was not updated, will
-    * be retained. When state was not updated for the specified interval of time, it will be cleared
-    * and removed.
-    *
-    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
-    * was the first data. This can result in previous results being overwritten.
-    *
-    * Note: [[setIdleStateRetentionTime(minTime: Time, maxTime: Time)]] allows to set a minimum and
-    * maximum time for state to be retained. This method is more efficient, because the system has
-    * to do less bookkeeping to identify the time at which state must be cleared.
-    *
-    * @param time The time interval for how long idle state is retained. Set to 0 (zero) to never
-    *             clean-up the state.
-    */
-  def setIdleStateRetentionTime(time: Time): StreamQueryConfig = {
-    setIdleStateRetentionTime(time, time)
-  }
-
-  /**
-    * Specifies a minimum and a maximum time interval for how long idle state, i.e., state which
-    * was not updated, will be retained.
-    * State will never be cleared until it was idle for less than the minimum time and will never
-    * be kept if it was idle for more than the maximum time.
-    *
-    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
-    * was the first data. This can result in previous results being overwritten.
-    *
-    * Set to 0 (zero) to never clean-up the state.
-    *
-    * @param minTime The minimum time interval for which idle state is retained. Set to 0 (zero) to
-    *                never clean-up the state.
-    * @param maxTime The maximum time interval for which idle state is retained. May not be smaller
-    *                than than minTime. Set to 0 (zero) to never clean-up the state.
-    */
-  def setIdleStateRetentionTime(minTime: Time, maxTime: Time): StreamQueryConfig = {
-    if (maxTime.toMilliseconds < minTime.toMilliseconds) {
-      throw new IllegalArgumentException("maxTime may not be smaller than minTime.")
-    }
-    minIdleStateRetentionTime = minTime.toMilliseconds
-    maxIdleStateRetentionTime = maxTime.toMilliseconds
-    this
-  }
-
-  def getMinIdleStateRetentionTime: Long = {
-    minIdleStateRetentionTime
-  }
-
-  def getMaxIdleStateRetentionTime: Long = {
-    maxIdleStateRetentionTime
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 c594d4c..d68da04 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
@@ -81,7 +81,7 @@ abstract class StreamTableEnvironment(
   // the naming pattern for internally registered tables.
   private val internalNamePattern = "^_DataStreamTable_[0-9]+$".r
 
-  def qConf: StreamQueryConfig = new StreamQueryConfig
+  def queryConfig: StreamQueryConfig = new StreamQueryConfig
 
   /**
     * Checks if the chosen table name is valid.
@@ -128,16 +128,16 @@ abstract class StreamTableEnvironment(
     *
     * @param table The [[Table]] to write.
     * @param sink The [[TableSink]] to write the [[Table]] to.
-    * @param qConfig The configuration for the query to generate.
+    * @param queryConfig The configuration for the query to generate.
     * @tparam T The expected type of the [[DataStream]] which represents the [[Table]].
     */
   override private[flink] def writeToSink[T](
       table: Table,
       sink: TableSink[T],
-      qConfig: QueryConfig): Unit = {
+      queryConfig: QueryConfig): Unit = {
 
     // Check query configuration
-    val sQConf = qConfig match {
+    val streamQueryConfig = queryConfig match {
       case streamConfig: StreamQueryConfig => streamConfig
       case _ =>
         throw new TableException("StreamQueryConfig required to configure stream query.")
@@ -150,7 +150,11 @@ abstract class StreamTableEnvironment(
         val outputType = sink.getOutputType
         // translate the Table into a DataStream and provide the type that the TableSink expects.
         val result: DataStream[T] =
-          translate(table, sQConf, updatesAsRetraction = true, withChangeFlag = true)(outputType)
+          translate(
+            table,
+            streamQueryConfig,
+            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]]])
@@ -176,7 +180,7 @@ abstract class StreamTableEnvironment(
           translate(
             optimizedPlan,
             table.getRelNode.getRowType,
-            sQConf,
+            streamQueryConfig,
             withChangeFlag = true)(outputType)
         // Give the DataStream to the TableSink to emit it.
         upsertSink.asInstanceOf[UpsertStreamTableSink[Any]]
@@ -196,7 +200,7 @@ abstract class StreamTableEnvironment(
           translate(
             optimizedPlan,
             table.getRelNode.getRowType,
-            sQConf,
+            streamQueryConfig,
             withChangeFlag = false)(outputType)
         // Give the DataStream to the TableSink to emit it.
         appendSink.asInstanceOf[AppendStreamTableSink[T]].emitDataStream(result)
@@ -566,7 +570,7 @@ 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 qConfig The configuration for the query to generate.
+    * @param queryConfig The configuration for the query to generate.
     * @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]].
@@ -575,12 +579,12 @@ abstract class StreamTableEnvironment(
     */
   protected def translate[A](
       table: Table,
-      qConfig: StreamQueryConfig,
+      queryConfig: StreamQueryConfig,
       updatesAsRetraction: Boolean,
       withChangeFlag: Boolean)(implicit tpe: TypeInformation[A]): DataStream[A] = {
     val relNode = table.getRelNode
     val dataStreamPlan = optimize(relNode, updatesAsRetraction)
-    translate(dataStreamPlan, relNode.getRowType, qConfig, withChangeFlag)
+    translate(dataStreamPlan, relNode.getRowType, queryConfig, withChangeFlag)
   }
 
   /**
@@ -589,7 +593,7 @@ 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 qConfig     The configuration for the query to generate.
+    * @param queryConfig     The configuration for the query to generate.
     * @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]].
@@ -598,7 +602,7 @@ abstract class StreamTableEnvironment(
   protected def translate[A](
       logicalPlan: RelNode,
       logicalType: RelDataType,
-      qConfig: StreamQueryConfig,
+      queryConfig: StreamQueryConfig,
       withChangeFlag: Boolean)
       (implicit tpe: TypeInformation[A]): DataStream[A] = {
 
@@ -610,7 +614,7 @@ abstract class StreamTableEnvironment(
     }
 
     // get CRow plan
-    val plan: DataStream[CRow] = translateToCRow(logicalPlan, qConfig)
+    val plan: DataStream[CRow] = translateToCRow(logicalPlan, queryConfig)
 
     // convert CRow to output type
     val conversion = if (withChangeFlag) {
@@ -642,16 +646,16 @@ abstract class StreamTableEnvironment(
     * Translates a logical [[RelNode]] plan into a [[DataStream]] of type [[CRow]].
     *
     * @param logicalPlan The logical plan to translate.
-    * @param qConfig The configuration for the query to generate.
+    * @param queryConfig  The configuration for the query to generate.
     * @return The [[DataStream]] of type [[CRow]].
     */
   protected def translateToCRow(
     logicalPlan: RelNode,
-    qConfig: StreamQueryConfig): DataStream[CRow] = {
+    queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     logicalPlan match {
       case node: DataStreamRel =>
-        node.translateToPlan(this, qConfig)
+        node.translateToPlan(this, queryConfig)
       case _ =>
         throw TableException("Cannot generate DataStream due to an invalid logical plan. " +
           "This is a bug and should not happen. Please file an issue.")
@@ -667,7 +671,7 @@ abstract class StreamTableEnvironment(
   def explain(table: Table): String = {
     val ast = table.getRelNode
     val optimizedPlan = optimize(ast, updatesAsRetraction = false)
-    val dataStream = translateToCRow(optimizedPlan, qConf)
+    val dataStream = translateToCRow(optimizedPlan, queryConfig)
 
     val env = dataStream.getExecutionEnvironment
     val jsonSqlPlan = env.getExecutionPlan

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 c3b5951..311986c 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
@@ -150,7 +150,7 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = {
-    toDataStream(table, clazz, qConf)
+    toDataStream(table, clazz, queryConfig)
   }
 
   /**
@@ -170,7 +170,7 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = {
-    toDataStream(table, typeInfo, qConf)
+    toDataStream(table, typeInfo, queryConfig)
   }
 
   /**
@@ -186,14 +186,17 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param clazz The class of the type of the resulting [[DataStream]].
-    * @param qConfig The configuration of the query to generate.
+    * @param queryConfig The configuration of the query to generate.
     * @tparam T The type of the resulting [[DataStream]].
     * @return The converted [[DataStream]].
     */
-  def toDataStream[T](table: Table, clazz: Class[T], qConfig: StreamQueryConfig): DataStream[T] = {
+  def toDataStream[T](
+      table: Table,
+      clazz: Class[T],
+      queryConfig: StreamQueryConfig): DataStream[T] = {
     val typeInfo = TypeExtractor.createTypeInfo(clazz)
     TableEnvironment.validateType(typeInfo)
-    translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
+    translate[T](table, queryConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
   }
 
   /**
@@ -209,16 +212,16 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param typeInfo The [[TypeInformation]] that specifies the type of the [[DataStream]].
-    * @param qConfig The configuration of the query to generate.
+    * @param queryConfig The configuration of the query to generate.
     * @tparam T The type of the resulting [[DataStream]].
     * @return The converted [[DataStream]].
     */
   def toDataStream[T](
       table: Table,
       typeInfo: TypeInformation[T],
-      qConfig: StreamQueryConfig): DataStream[T] = {
+      queryConfig: StreamQueryConfig): DataStream[T] = {
     TableEnvironment.validateType(typeInfo)
-    translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
+    translate[T](table, queryConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
   }
 
   /**
@@ -242,7 +245,7 @@ class StreamTableEnvironment(
       table: Table,
       clazz: Class[T]): DataStream[JTuple2[JBool, T]] = {
 
-    toRetractStream(table, clazz, qConf)
+    toRetractStream(table, clazz, queryConfig)
   }
 
   /**
@@ -266,7 +269,7 @@ class StreamTableEnvironment(
       table: Table,
       typeInfo: TypeInformation[T]): DataStream[JTuple2[JBool, T]] = {
 
-    toRetractStream(table, typeInfo, qConf)
+    toRetractStream(table, typeInfo, queryConfig)
   }
 
   /**
@@ -283,21 +286,21 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param clazz The class of the requested record type.
-    * @param qConfig The configuration of the query to generate.
+    * @param queryConfig The configuration of the query to generate.
     * @tparam T The type of the requested record type.
     * @return The converted [[DataStream]].
     */
   def toRetractStream[T](
       table: Table,
       clazz: Class[T],
-      qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = {
+      queryConfig: StreamQueryConfig): 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,
-      qConfig,
+      queryConfig,
       updatesAsRetraction = true,
       withChangeFlag = true)(resultType)
   }
@@ -316,14 +319,14 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param typeInfo The [[TypeInformation]] of the requested record type.
-    * @param qConfig The configuration of the query to generate.
+    * @param queryConfig The configuration of the query to generate.
     * @tparam T The type of the requested record type.
     * @return The converted [[DataStream]].
     */
   def toRetractStream[T](
       table: Table,
       typeInfo: TypeInformation[T],
-      qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = {
+      queryConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = {
 
     TableEnvironment.validateType(typeInfo)
     val resultTypeInfo = new TupleTypeInfo[JTuple2[JBool, T]](
@@ -332,7 +335,7 @@ class StreamTableEnvironment(
     )
     translate[JTuple2[JBool, T]](
       table,
-      qConfig,
+      queryConfig,
       updatesAsRetraction = true,
       withChangeFlag = true)(resultTypeInfo)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala
new file mode 100644
index 0000000..c8fbab7
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import _root_.java.io.Serializable
+import org.apache.flink.api.common.time.Time
+
+class QueryConfig private[table] extends Serializable {}
+
+/**
+  * The [[BatchQueryConfig]] holds parameters to configure the behavior of batch queries.
+  */
+class BatchQueryConfig private[table] extends QueryConfig
+
+/**
+  * The [[StreamQueryConfig]] holds parameters to configure the behavior of streaming queries.
+  *
+  * An empty [[StreamQueryConfig]] can be generated using the [[StreamTableEnvironment.queryConfig]]
+  * method.
+  */
+class StreamQueryConfig private[table] extends QueryConfig {
+
+  /**
+    * The minimum time until state which was not updated will be retained.
+    * State might be cleared and removed if it was not updated for the defined period of time.
+    */
+  private var minIdleStateRetentionTime: Long = Long.MinValue
+
+  /**
+    * The maximum time until state which was not updated will be retained.
+    * State will be cleared and removed if it was not updated for the defined period of time.
+    */
+  private var maxIdleStateRetentionTime: Long = Long.MinValue
+
+  /**
+    * Specifies the time interval for how long idle state, i.e., state which was not updated, will
+    * be retained. When state was not updated for the specified interval of time, it will be cleared
+    * and removed.
+    *
+    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
+    * was the first data. This can result in previous results being overwritten.
+    *
+    * Note: [[withIdleStateRetentionTime(minTime: Time, maxTime: Time)]] allows to set a minimum and
+    * maximum time for state to be retained. This method is more efficient, because the system has
+    * to do less bookkeeping to identify the time at which state must be cleared.
+    *
+    * @param time The time interval for how long idle state is retained. Set to 0 (zero) to never
+    *             clean-up the state.
+    */
+  def withIdleStateRetentionTime(time: Time): StreamQueryConfig = {
+    withIdleStateRetentionTime(time, time)
+  }
+
+  /**
+    * Specifies a minimum and a maximum time interval for how long idle state, i.e., state which
+    * was not updated, will be retained.
+    * State will never be cleared until it was idle for less than the minimum time and will never
+    * be kept if it was idle for more than the maximum time.
+    *
+    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
+    * was the first data. This can result in previous results being overwritten.
+    *
+    * Set to 0 (zero) to never clean-up the state.
+    *
+    * @param minTime The minimum time interval for which idle state is retained. Set to 0 (zero) to
+    *                never clean-up the state.
+    * @param maxTime The maximum time interval for which idle state is retained. May not be smaller
+    *                than than minTime. Set to 0 (zero) to never clean-up the state.
+    */
+  def withIdleStateRetentionTime(minTime: Time, maxTime: Time): StreamQueryConfig = {
+    if (maxTime.toMilliseconds < minTime.toMilliseconds) {
+      throw new IllegalArgumentException("maxTime may not be smaller than minTime.")
+    }
+    minIdleStateRetentionTime = minTime.toMilliseconds
+    maxIdleStateRetentionTime = maxTime.toMilliseconds
+    this
+  }
+
+  def getMinIdleStateRetentionTime: Long = {
+    minIdleStateRetentionTime
+  }
+
+  def getMaxIdleStateRetentionTime: Long = {
+    maxIdleStateRetentionTime
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 56f7d55..8c6b273 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
@@ -143,7 +143,7 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T: TypeInformation](table: Table): DataStream[T] = {
-    toDataStream(table, qConf)
+    toDataStream(table, queryConfig)
   }
 
   /**
@@ -158,14 +158,16 @@ class StreamTableEnvironment(
     * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match.
     *
     * @param table The [[Table]] to convert.
-    * @param qConfig The configuration of the query to generate.
+    * @param queryConfig The configuration of the query to generate.
     * @tparam T The type of the resulting [[DataStream]].
     * @return The converted [[DataStream]].
     */
-  def toDataStream[T: TypeInformation](table: Table, qConfig: StreamQueryConfig): DataStream[T] = {
+  def toDataStream[T: TypeInformation](
+    table: Table,
+    queryConfig: StreamQueryConfig): DataStream[T] = {
     val returnType = createTypeInformation[T]
-    asScalaStream(
-      translate(table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(returnType))
+    asScalaStream(translate(
+      table, queryConfig, updatesAsRetraction = false, withChangeFlag = false)(returnType))
   }
 
 /**
@@ -180,7 +182,7 @@ class StreamTableEnvironment(
   * @return The converted [[DataStream]].
   */
   def toRetractStream[T: TypeInformation](table: Table): DataStream[(Boolean, T)] = {
-    toRetractStream(table, qConf)
+    toRetractStream(table, queryConfig)
   }
 
   /**
@@ -191,16 +193,16 @@ class StreamTableEnvironment(
     * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message.
     *
     * @param table The [[Table]] to convert.
-    * @param qConfig The configuration of the query to generate.
+    * @param queryConfig The configuration of the query to generate.
     * @tparam T The type of the requested data type.
     * @return The converted [[DataStream]].
     */
   def toRetractStream[T: TypeInformation](
       table: Table,
-      qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = {
+      queryConfig: StreamQueryConfig): DataStream[(Boolean, T)] = {
     val returnType = createTypeInformation[(Boolean, T)]
     asScalaStream(
-      translate(table, qConfig, updatesAsRetraction = true, withChangeFlag = true)(returnType))
+      translate(table, queryConfig, updatesAsRetraction = true, withChangeFlag = true)(returnType))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 966b42f..9874a9e 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
@@ -59,12 +59,12 @@ class TableConversions(table: Table) {
 
   /** Converts the [[Table]] to a [[DataStream]] of the specified type.
     *
-    * @param qConfig The configuration for the generated query.
+    * @param queryConfig The configuration for the generated query.
     */
-  def toDataStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[T] = {
+  def toDataStream[T: TypeInformation](queryConfig: StreamQueryConfig): DataStream[T] = {
     table.tableEnv match {
       case tEnv: ScalaStreamTableEnv =>
-        tEnv.toDataStream(table, qConfig)
+        tEnv.toDataStream(table, queryConfig)
       case _ =>
         throw new TableException(
           "Only tables that originate from Scala DataStreams " +
@@ -97,14 +97,15 @@ class TableConversions(table: Table) {
     *
     * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message.
     *
-    * @param qConfig The configuration for the generated query.
+    * @param queryConfig The configuration for the generated query.
     *
     */
-  def toRetractStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = {
+  def toRetractStream[T: TypeInformation](
+      queryConfig: StreamQueryConfig): DataStream[(Boolean, T)] = {
 
     table.tableEnv match {
       case tEnv: ScalaStreamTableEnv =>
-        tEnv.toRetractStream(table, qConfig)
+        tEnv.toRetractStream(table, queryConfig)
       case _ =>
         throw new TableException(
           "Only tables that originate from Scala DataStreams " +

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 5a2eb1c..ca61c65 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
@@ -764,13 +764,13 @@ class Table(
     */
   def writeToSink[T](sink: TableSink[T]): Unit = {
 
-    def qConfig = this.tableEnv match {
-      case s: StreamTableEnvironment => s.qConf
+    def queryConfig = this.tableEnv match {
+      case s: StreamTableEnvironment => s.queryConfig
       case b: BatchTableEnvironment => new BatchQueryConfig
       case _ => null
     }
 
-    writeToSink(sink, qConfig)
+    writeToSink(sink, queryConfig)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 0e377b5..5f270f6 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
@@ -85,11 +85,12 @@ class DataStreamCalc(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
-    val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val inputDataStream =
+      getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
     val inputRowType = inputDataStream.getType.asInstanceOf[CRowTypeInfo].rowType
 
     val generator = new CodeGenerator(config, false, inputRowType)

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 cbd818a..5b32b10 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
@@ -84,12 +84,12 @@ class DataStreamCorrelate(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
     // we do not need to specify input type
-    val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
     val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo]
 
     val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 f01b24a..e5d8088 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
@@ -106,16 +106,16 @@ class DataStreamGroupAggregate(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
-    if (qConfig.getMinIdleStateRetentionTime < 0 || qConfig.getMaxIdleStateRetentionTime < 0) {
+    if (groupings.length > 0 && queryConfig.getMinIdleStateRetentionTime < 0) {
       LOG.warn(
         "No state retention interval configured for a query which accumulates state. " +
         "Please provide a query configuration with valid retention interval to prevent excessive " +
-          "state size. You may specify a retention time of 0 to not clean up the state.")
+        "state size. You may specify a retention time of 0 to not clean up the state.")
     }
 
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
 
     val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
       new CalcitePair[AggregateCall, String](
@@ -149,7 +149,7 @@ class DataStreamGroupAggregate(
       inputSchema.logicalType,
       inputSchema.physicalFieldTypeInfo,
       groupings,
-      qConfig,
+      queryConfig,
       DataStreamRetractionRules.isAccRetract(this),
       DataStreamRetractionRules.isAccRetract(getInput))
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 d2aaad0..2a71592 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
@@ -109,9 +109,9 @@ class DataStreamGroupWindowAggregate(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
 
     val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
       new CalcitePair[AggregateCall, String](

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 8e97884..a9fbf02 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
@@ -36,6 +36,7 @@ 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}
+import org.slf4j.LoggerFactory
 
 class DataStreamOverAggregate(
     logicWindow: Window,
@@ -47,6 +48,7 @@ class DataStreamOverAggregate(
   extends SingleRel(cluster, traitSet, inputNode)
   with OverAggregate
   with DataStreamRel {
+  private val LOG = LoggerFactory.getLogger(this.getClass)
 
   override def deriveRowType(): RelDataType = schema.logicalType
 
@@ -90,7 +92,7 @@ class DataStreamOverAggregate(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     if (logicWindow.groups.size > 1) {
       throw new TableException(
@@ -112,10 +114,23 @@ class DataStreamOverAggregate(
         "Unsupported use of OVER windows. The window can only be ordered in ASCENDING mode.")
     }
 
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
 
     val consumeRetraction = DataStreamRetractionRules.isAccRetract(input)
 
+    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.")
+    }
+
+    if (!logicWindow.groups.get(0).keys.isEmpty && queryConfig.getMinIdleStateRetentionTime < 0) {
+      LOG.warn(
+        "No state retention interval configured for a query which accumulates state. " +
+        "Please provide a query configuration with valid retention interval to prevent " +
+        "excessive state size. You may specify a retention time of 0 to not clean up the state.")
+    }
+
     val generator = new CodeGenerator(
       tableEnv.getConfig,
       false,
@@ -126,18 +141,13 @@ 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
         if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
           // unbounded OVER window
           createUnboundedAndCurrentRowOverWindow(
+            queryConfig,
             generator,
             inputDS,
             isRowTimeType = false,
@@ -145,8 +155,10 @@ class DataStreamOverAggregate(
         } else if (
           overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded &&
             overWindow.upperBound.isCurrentRow) {
+
           // bounded OVER window
           createBoundedAndCurrentRowOverWindow(
+            queryConfig,
             generator,
             inputDS,
             isRowTimeType = false,
@@ -162,6 +174,7 @@ class DataStreamOverAggregate(
           overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
           // unbounded OVER window
           createUnboundedAndCurrentRowOverWindow(
+            queryConfig,
             generator,
             inputDS,
             isRowTimeType = true,
@@ -169,6 +182,7 @@ class DataStreamOverAggregate(
         } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) {
           // bounded OVER window
           createBoundedAndCurrentRowOverWindow(
+            queryConfig,
             generator,
             inputDS,
             isRowTimeType = true,
@@ -185,6 +199,7 @@ class DataStreamOverAggregate(
   }
 
   def createUnboundedAndCurrentRowOverWindow(
+    queryConfig: StreamQueryConfig,
     generator: CodeGenerator,
     inputDS: DataStream[CRow],
     isRowTimeType: Boolean,
@@ -210,6 +225,7 @@ class DataStreamOverAggregate(
       inputSchema.physicalType,
       inputSchema.physicalTypeInfo,
       inputSchema.physicalFieldTypeInfo,
+      queryConfig,
       isRowTimeType,
       partitionKeys.nonEmpty,
       isRowsClause)
@@ -242,6 +258,7 @@ class DataStreamOverAggregate(
   }
 
   def createBoundedAndCurrentRowOverWindow(
+    queryConfig: StreamQueryConfig,
     generator: CodeGenerator,
     inputDS: DataStream[CRow],
     isRowTimeType: Boolean,
@@ -269,6 +286,7 @@ class DataStreamOverAggregate(
       inputSchema.physicalTypeInfo,
       inputSchema.physicalFieldTypeInfo,
       precedingOffset,
+      queryConfig,
       isRowsClause,
       isRowTimeType
     )

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 6f6edf7..65d336f 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
@@ -29,12 +29,12 @@ trait DataStreamRel extends FlinkRelNode {
     * Translates the FlinkRelNode into a Flink operator.
     *
     * @param tableEnv The [[StreamTableEnvironment]] of the translated Table.
-    * @param qConfig The configuration for the query to generate.
+    * @param queryConfig The configuration for the query to generate.
     * @return DataStream of type [[CRow]]
     */
   def translateToPlan(
     tableEnv: StreamTableEnvironment,
-    qConfig: StreamQueryConfig): DataStream[CRow]
+    queryConfig: StreamQueryConfig): 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/6b61d153/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 e64bf0f..424c6a2 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
@@ -56,7 +56,7 @@ class DataStreamScan(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
     val inputDataStream: DataStream[Any] = dataStreamTable.dataStream

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 6cc7396..6f4980a 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
@@ -60,10 +60,10 @@ class DataStreamUnion(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
-    val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
-    val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+    val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
     leftDataSet.union(rightDataSet)
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 ba6b025..d7c490f 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
@@ -58,7 +58,7 @@ class DataStreamValues(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 225f23f..51e609f 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
@@ -100,7 +100,7 @@ class StreamTableSourceScan(
 
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
-      qConfig: StreamQueryConfig): DataStream[CRow] = {
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
     val inputDataStream = tableSource.getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]]

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 27392c7..8073959 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
@@ -77,6 +77,7 @@ object AggregateUtil {
       inputType: RelDataType,
       inputTypeInfo: TypeInformation[Row],
       inputFieldTypeInfo: Seq[TypeInformation[_]],
+      queryConfig: StreamQueryConfig,
       isRowTimeType: Boolean,
       isPartitioned: Boolean,
       isRowsClause: Boolean)
@@ -117,23 +118,27 @@ object AggregateUtil {
         new RowTimeUnboundedRowsOver(
           genFunction,
           aggregationStateType,
-          CRowTypeInfo(inputTypeInfo))
+          CRowTypeInfo(inputTypeInfo),
+          queryConfig)
       } else {
         // RANGE unbounded over process function
         new RowTimeUnboundedRangeOver(
           genFunction,
           aggregationStateType,
-          CRowTypeInfo(inputTypeInfo))
+          CRowTypeInfo(inputTypeInfo),
+          queryConfig)
       }
     } else {
       if (isPartitioned) {
         new ProcTimeUnboundedPartitionedOver(
           genFunction,
-          aggregationStateType)
+          aggregationStateType,
+          queryConfig)
       } else {
         new ProcTimeUnboundedNonPartitionedOver(
           genFunction,
-          aggregationStateType)
+          aggregationStateType,
+          queryConfig)
       }
     }
   }
@@ -155,7 +160,7 @@ object AggregateUtil {
       inputRowType: RelDataType,
       inputFieldTypes: Seq[TypeInformation[_]],
       groupings: Array[Int],
-      qConfig: StreamQueryConfig,
+      queryConfig: StreamQueryConfig,
       generateRetraction: Boolean,
       consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = {
 
@@ -192,7 +197,7 @@ object AggregateUtil {
       genFunction,
       aggregationStateType,
       generateRetraction,
-      qConfig)
+      queryConfig)
 
   }
 
@@ -217,6 +222,7 @@ object AggregateUtil {
       inputTypeInfo: TypeInformation[Row],
       inputFieldTypeInfo: Seq[TypeInformation[_]],
       precedingOffset: Long,
+      queryConfig: StreamQueryConfig,
       isRowsClause: Boolean,
       isRowTimeType: Boolean)
     : ProcessFunction[CRow, CRow] = {
@@ -258,15 +264,15 @@ object AggregateUtil {
           genFunction,
           aggregationStateType,
           inputRowType,
-          precedingOffset
-        )
+          precedingOffset,
+          queryConfig)
       } else {
         new RowTimeBoundedRangeOver(
           genFunction,
           aggregationStateType,
           inputRowType,
-          precedingOffset
-        )
+          precedingOffset,
+          queryConfig)
       }
     } else {
       if (isRowsClause) {
@@ -274,13 +280,15 @@ object AggregateUtil {
           genFunction,
           precedingOffset,
           aggregationStateType,
-          inputRowType)
+          inputRowType,
+          queryConfig)
       } else {
         new ProcTimeBoundedRangeOver(
           genFunction,
           precedingOffset,
           aggregationStateType,
-          inputRowType)
+          inputRowType,
+          queryConfig)
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 84fee87..57ea86e 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
@@ -41,19 +41,13 @@ class GroupAggProcessFunction(
     private val genAggregations: GeneratedAggregationsFunction,
     private val aggregationStateType: RowTypeInfo,
     private val generateRetraction: Boolean,
-    private val qConfig: StreamQueryConfig)
-  extends ProcessFunction[CRow, CRow]
+    private val queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
 
   val LOG: Logger = LoggerFactory.getLogger(this.getClass)
   private var function: GeneratedAggregations = _
 
-  private val minRetentionTime = qConfig.getMinIdleStateRetentionTime
-  private val maxRetentionTime = qConfig.getMaxIdleStateRetentionTime
-  private val stateCleaningEnabled = minRetentionTime > 1 && maxRetentionTime > 1
-  // interval in which clean-up timers are registered
-  private val cleanupTimerInterval = maxRetentionTime - minRetentionTime
-
   private var newRow: CRow = _
   private var prevRow: CRow = _
   private var firstRow: Boolean = _
@@ -61,8 +55,6 @@ class GroupAggProcessFunction(
   private var state: ValueState[Row] = _
   // counts the number of added and retracted input records
   private var cntState: ValueState[JLong] = _
-  // holds the latest registered cleanup timer
-  private var cleanupTimeState: ValueState[JLong] = _
 
   override def open(config: Configuration) {
     LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " +
@@ -84,11 +76,7 @@ class GroupAggProcessFunction(
       new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG)
     cntState = getRuntimeContext.getState(inputCntDescriptor)
 
-    if (stateCleaningEnabled) {
-      val inputCntDescriptor: ValueStateDescriptor[JLong] =
-        new ValueStateDescriptor[JLong]("GroupAggregateCleanupTime", Types.LONG)
-      cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor)
-    }
+    initCleanupTimeState("GroupAggregateCleanupTime")
   }
 
   override def processElement(
@@ -96,22 +84,9 @@ class GroupAggProcessFunction(
       ctx: ProcessFunction[CRow, CRow]#Context,
       out: Collector[CRow]): Unit = {
 
-    if (stateCleaningEnabled) {
-
-      val currentTime = ctx.timerService().currentProcessingTime()
-      val earliestCleanup = currentTime + minRetentionTime
-
-      // last registered timer
-      val lastCleanupTime = cleanupTimeState.value()
-
-      if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) {
-        // we need to register a new timer
-        val cleanupTime = earliestCleanup + cleanupTimerInterval
-        // register timer and remember clean-up time
-        ctx.timerService().registerProcessingTimeTimer(cleanupTime)
-        cleanupTimeState.update(cleanupTime)
-      }
-    }
+    val currentTime = ctx.timerService().currentProcessingTime()
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, currentTime)
 
     val input = inputC.row
 
@@ -182,11 +157,8 @@ class GroupAggProcessFunction(
       ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
       out: Collector[CRow]): Unit = {
 
-    if (timestamp == cleanupTimeState.value()) {
-      // clear all state
-      this.state.clear()
-      this.cntState.clear()
-      this.cleanupTimeState.clear()
+    if (needToCleanupState(timestamp)) {
+      cleanupState(state, cntState)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 3fb506f..d50912c 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,6 +31,7 @@ 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.api.StreamQueryConfig
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
@@ -48,9 +49,11 @@ class ProcTimeBoundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
     precedingTimeBoundary: Long,
     aggregatesTypeInfo: RowTypeInfo,
-    inputType: TypeInformation[CRow])
-  extends ProcessFunction[CRow, CRow]
+    inputType: TypeInformation[CRow],
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
+
   private var output: CRow = _
   private var accumulatorState: ValueState[Row] = _
   private var rowMapState: MapState[Long, JList[Row]] = _
@@ -81,6 +84,8 @@ class ProcTimeBoundedRangeOver(
     val stateDescriptor: ValueStateDescriptor[Row] =
       new ValueStateDescriptor[Row]("overState", aggregatesTypeInfo)
     accumulatorState = getRuntimeContext.getState(stateDescriptor)
+
+    initCleanupTimeState("ProcTimeBoundedRangeOverCleanupTime")
   }
 
   override def processElement(
@@ -89,6 +94,9 @@ class ProcTimeBoundedRangeOver(
     out: Collector[CRow]): Unit = {
 
     val currentTime = ctx.timerService.currentProcessingTime
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, currentTime)
+
     // buffer the event incoming event
 
     // add current element to the window list of elements with corresponding timestamp
@@ -109,7 +117,15 @@ class ProcTimeBoundedRangeOver(
     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
+    if (needToCleanupState(timestamp)) {
+      // clean up and return
+      cleanupState(rowMapState, accumulatorState)
+      return
+    }
+
+    // we consider the original timestamp of events
+    // that have registered this time trigger 1 ms ago
+
     val currentTime = timestamp - 1
     var i = 0
 
@@ -153,7 +169,8 @@ class ProcTimeBoundedRangeOver(
 
     // get the list of elements of current proctime
     val currentElements = rowMapState.get(currentTime)
-    // add current elements to aggregator. Multiple elements might have arrived in the same proctime
+    // add current elements to aggregator. Multiple elements might
+    // have arrived in the same proctime
     // the same accumulator value will be computed for all elements
     var iElemenets = 0
     while (iElemenets < currentElements.size()) {
@@ -178,7 +195,6 @@ class ProcTimeBoundedRangeOver(
 
     // update the value of accumulators for future incremental computation
     accumulatorState.update(accumulators)
-
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 0c7f44e..e388c93 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,6 +33,7 @@ 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.api.StreamQueryConfig
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
@@ -49,8 +50,9 @@ class ProcTimeBoundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     precedingOffset: Long,
     aggregatesTypeInfo: RowTypeInfo,
-    inputType: TypeInformation[CRow])
-  extends ProcessFunction[CRow, CRow]
+    inputType: TypeInformation[CRow],
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
 
   Preconditions.checkArgument(precedingOffset > 0)
@@ -99,6 +101,8 @@ class ProcTimeBoundedRowsOver(
     val smallestTimestampDescriptor : ValueStateDescriptor[Long] =
        new ValueStateDescriptor[Long]("smallestTSState", classOf[Long])
     smallestTsState = getRuntimeContext.getState(smallestTimestampDescriptor)
+
+    initCleanupTimeState("ProcTimeBoundedRowsOverCleanupTime")
   }
 
   override def processElement(
@@ -110,6 +114,9 @@ class ProcTimeBoundedRowsOver(
 
     val currentTime = ctx.timerService.currentProcessingTime
 
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, currentTime)
+
     // initialize state for the processed element
     var accumulators = accumulatorState.value
     if (accumulators == null) {
@@ -180,4 +187,13 @@ class ProcTimeBoundedRowsOver(
     out.collect(output)
   }
 
+  override def onTimer(
+    timestamp: Long,
+    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+    out: Collector[CRow]): Unit = {
+
+    if (needToCleanupState(timestamp)) {
+      cleanupState(rowMapState, accumulatorState, counterState, smallestTsState)
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 8a23132..2a6c9c8 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,6 +23,7 @@ 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.table.api.StreamQueryConfig
 import org.apache.flink.util.Collector
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
@@ -37,8 +38,9 @@ import org.slf4j.LoggerFactory
   */
 class ProcTimeUnboundedNonPartitionedOver(
     genAggregations: GeneratedAggregationsFunction,
-    aggregationStateType: RowTypeInfo)
-  extends ProcessFunction[CRow, CRow]
+    aggregationStateType: RowTypeInfo,
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with CheckpointedFunction
     with Compiler[GeneratedAggregations] {
 
@@ -68,12 +70,16 @@ class ProcTimeUnboundedNonPartitionedOver(
         accumulators = function.createAccumulators()
       }
     }
+    initCleanupTimeState("ProcTimeUnboundedNonPartitionedOverCleanupTime")
   }
 
   override def processElement(
       inputC: CRow,
       ctx: ProcessFunction[CRow, CRow]#Context,
       out: Collector[CRow]): Unit = {
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
 
     val input = inputC.row
 
@@ -85,6 +91,16 @@ class ProcTimeUnboundedNonPartitionedOver(
     out.collect(output)
   }
 
+  override def onTimer(
+    timestamp: Long,
+    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+    out: Collector[CRow]): Unit = {
+
+    if (needToCleanupState(timestamp)) {
+      cleanupState(state)
+    }
+  }
+
   override def snapshotState(context: FunctionSnapshotContext): Unit = {
     state.clear()
     if (null != accumulators) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 847c1bf..97f0ad7 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,6 +24,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.StreamQueryConfig
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.apache.flink.table.runtime.types.CRow
 import org.slf4j.LoggerFactory
@@ -36,8 +37,9 @@ import org.slf4j.LoggerFactory
   */
 class ProcTimeUnboundedPartitionedOver(
     genAggregations: GeneratedAggregationsFunction,
-    aggregationStateType: RowTypeInfo)
-  extends ProcessFunction[CRow, CRow]
+    aggregationStateType: RowTypeInfo,
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
 
   private var output: CRow = _
@@ -59,6 +61,8 @@ class ProcTimeUnboundedPartitionedOver(
     val stateDescriptor: ValueStateDescriptor[Row] =
       new ValueStateDescriptor[Row]("overState", aggregationStateType)
     state = getRuntimeContext.getState(stateDescriptor)
+
+    initCleanupTimeState("ProcTimeUnboundedPartitionedOverCleanupTime")
   }
 
   override def processElement(
@@ -66,6 +70,9 @@ class ProcTimeUnboundedPartitionedOver(
     ctx: ProcessFunction[CRow, CRow]#Context,
     out: Collector[CRow]): Unit = {
 
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
     val input = inputC.row
 
     var accumulators = state.value()
@@ -83,4 +90,13 @@ class ProcTimeUnboundedPartitionedOver(
     out.collect(output)
   }
 
+  override def onTimer(
+    timestamp: Long,
+    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+    out: Collector[CRow]): Unit = {
+
+    if (needToCleanupState(timestamp)) {
+      cleanupState(state)
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala
new file mode 100644
index 0000000..292fd3b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala
@@ -0,0 +1,85 @@
+/*
+ * 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 java.lang.{Long => JLong}
+
+import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
+import org.apache.flink.api.common.state.State
+import org.apache.flink.streaming.api.TimeDomain
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.api.{StreamQueryConfig, Types}
+
+abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryConfig)
+  extends ProcessFunction[IN, OUT]{
+
+  protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
+  protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
+  protected val stateCleaningEnabled: Boolean = minRetentionTime > 1
+
+  // holds the latest registered cleanup timer
+  private var cleanupTimeState: ValueState[JLong] = _
+
+  protected def initCleanupTimeState(stateName: String) {
+    if (stateCleaningEnabled) {
+      val inputCntDescriptor: ValueStateDescriptor[JLong] =
+        new ValueStateDescriptor[JLong](stateName, Types.LONG)
+      cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor)
+    }
+  }
+
+  protected def registerProcessingCleanupTimer(
+    ctx: ProcessFunction[IN, OUT]#Context,
+    currentTime: Long): Unit = {
+    if (stateCleaningEnabled) {
+
+      // last registered timer
+      val curCleanupTime = cleanupTimeState.value()
+
+      // check if a cleanup timer is registered and
+      // that the current cleanup timer won't delete state we need to keep
+      if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) {
+        // we need to register a new (later) timer
+        val cleanupTime = currentTime + maxRetentionTime
+        // register timer and remember clean-up time
+        ctx.timerService().registerProcessingTimeTimer(cleanupTime)
+        cleanupTimeState.update(cleanupTime)
+      }
+    }
+  }
+
+  protected def isProcessingTimeTimer(ctx: OnTimerContext): Boolean = {
+    ctx.timeDomain() == TimeDomain.PROCESSING_TIME
+  }
+
+  protected def needToCleanupState(timestamp: Long): Boolean = {
+    if (stateCleaningEnabled) {
+      val cleanupTime = cleanupTimeState.value()
+      // check that the triggered timer is the last registered processing time timer.
+      null != cleanupTime && timestamp == cleanupTime
+    } else {
+      false
+    }
+  }
+
+  protected def cleanupState(states: State*): Unit = {
+    // clear all state
+    states.foreach(_.clear())
+    this.cleanupTimeState.clear()
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 4020d44..65edf6d 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
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 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.api.StreamQueryConfig
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.apache.flink.types.Row
@@ -42,8 +43,9 @@ class RowTimeBoundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
     aggregationStateType: RowTypeInfo,
     inputRowType: CRowTypeInfo,
-    precedingOffset: Long)
-  extends ProcessFunction[CRow, CRow]
+    precedingOffset: Long,
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
   Preconditions.checkNotNull(aggregationStateType)
   Preconditions.checkNotNull(precedingOffset)
@@ -97,6 +99,8 @@ class RowTimeBoundedRangeOver(
         valueTypeInformation)
 
     dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+    initCleanupTimeState("RowTimeBoundedRangeOverCleanupTime")
   }
 
   override def processElement(
@@ -106,6 +110,9 @@ class RowTimeBoundedRangeOver(
 
     val input = inputC.row
 
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
     // triggering timestamp for trigger calculation
     val triggeringTs = ctx.timestamp
 
@@ -131,6 +138,34 @@ class RowTimeBoundedRangeOver(
     timestamp: Long,
     ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
     out: Collector[CRow]): Unit = {
+
+    if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) {
+      if (needToCleanupState(timestamp)) {
+
+        val keysIt = dataState.keys.iterator()
+        val lastProcessedTime = lastTriggeringTsState.value
+
+        // is data left which has not been processed yet?
+        var noRecordsToProcess = true
+        while (keysIt.hasNext && noRecordsToProcess) {
+          if (keysIt.next() > lastProcessedTime) {
+            noRecordsToProcess = false
+          }
+        }
+
+        if (noRecordsToProcess) {
+          // we clean the state
+          cleanupState(dataState, accumulatorState, lastTriggeringTsState)
+        } else {
+          // There are records left to process because a watermark has not been received yet.
+          // This would only happen if the input stream has stopped. So we don't need to clean up.
+          // We leave the state as it is and schedule a new cleanup timer
+          registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+        }
+      }
+      return
+    }
+
     // gets all window data from state for the calculation
     val inputs: JList[Row] = dataState.get(timestamp)
 
@@ -196,8 +231,11 @@ class RowTimeBoundedRangeOver(
 
       // update state
       accumulatorState.update(accumulators)
-      lastTriggeringTsState.update(timestamp)
     }
+    lastTriggeringTsState.update(timestamp)
+
+    // update cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
   }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 5ec6ec7..395ae39 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
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 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.api.StreamQueryConfig
 import org.apache.flink.types.Row
 import org.apache.flink.util.{Collector, Preconditions}
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
@@ -43,8 +44,9 @@ class RowTimeBoundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     aggregationStateType: RowTypeInfo,
     inputRowType: CRowTypeInfo,
-    precedingOffset: Long)
-  extends ProcessFunction[CRow, CRow]
+    precedingOffset: Long,
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
 
   Preconditions.checkNotNull(aggregationStateType)
@@ -106,6 +108,8 @@ class RowTimeBoundedRowsOver(
         valueTypeInformation)
 
     dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+    initCleanupTimeState("RowTimeBoundedRowsOverCleanupTime")
   }
 
   override def processElement(
@@ -115,6 +119,9 @@ class RowTimeBoundedRowsOver(
 
     val input = inputC.row
 
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
     // triggering timestamp for trigger calculation
     val triggeringTs = ctx.timestamp
 
@@ -141,6 +148,33 @@ class RowTimeBoundedRowsOver(
     ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
     out: Collector[CRow]): Unit = {
 
+    if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) {
+      if (needToCleanupState(timestamp)) {
+
+        val keysIt = dataState.keys.iterator()
+        val lastProcessedTime = lastTriggeringTsState.value
+
+        // is data left which has not been processed yet?
+        var noRecordsToProcess = true
+        while (keysIt.hasNext && noRecordsToProcess) {
+          if (keysIt.next() > lastProcessedTime) {
+            noRecordsToProcess = false
+          }
+        }
+
+        if (noRecordsToProcess) {
+          // We clean the state
+          cleanupState(dataState, accumulatorState, dataCountState, lastTriggeringTsState)
+        } else {
+          // There are records left to process because a watermark has not been received yet.
+          // This would only happen if the input stream has stopped. So we don't need to clean up.
+          // We leave the state as it is and schedule a new cleanup timer
+          registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+        }
+      }
+      return
+    }
+
     // gets all window data from state for the calculation
     val inputs: JList[Row] = dataState.get(timestamp)
 
@@ -220,6 +254,9 @@ class RowTimeBoundedRowsOver(
     }
 
     lastTriggeringTsState.update(timestamp)
+
+    // update cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
   }
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6b61d153/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 3e2a811..741d2b4 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,6 +28,7 @@ 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.api.StreamQueryConfig
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 import org.slf4j.LoggerFactory
@@ -43,8 +44,9 @@ import org.slf4j.LoggerFactory
 abstract class RowTimeUnboundedOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType: TypeInformation[Row],
-    inputType: TypeInformation[CRow])
-  extends ProcessFunction[CRow, CRow]
+    inputType: TypeInformation[CRow],
+    queryConfig: StreamQueryConfig)
+  extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
     with Compiler[GeneratedAggregations] {
 
   protected var output: CRow = _
@@ -83,6 +85,8 @@ abstract class RowTimeUnboundedOver(
       new MapStateDescriptor[Long, JList[Row]]("rowmapstate",
         BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo)
     rowMapState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+    initCleanupTimeState("RowTimeUnboundedOverCleanupTime")
   }
 
   /**
@@ -101,6 +105,9 @@ abstract class RowTimeUnboundedOver(
 
     val input = inputC.row
 
+    // register state-cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
     val timestamp = ctx.timestamp()
     val curWatermark = ctx.timerService().currentWatermark()
 
@@ -133,6 +140,24 @@ abstract class RowTimeUnboundedOver(
       ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
       out: Collector[CRow]): Unit = {
 
+    if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) {
+      if (needToCleanupState(timestamp)) {
+
+        // we check whether there are still records which have not been processed yet
+        val noRecordsToProcess = !rowMapState.keys.iterator().hasNext
+        if (noRecordsToProcess) {
+          // we clean the state
+          cleanupState(rowMapState, accumulatorState)
+        } else {
+          // There are records left to process because a watermark has not been received yet.
+          // This would only happen if the input stream has stopped. So we don't need to clean up.
+          // We leave the state as it is and schedule a new cleanup timer
+          registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+        }
+      }
+      return
+    }
+
     Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[CRow]])
     val collector = out.asInstanceOf[TimestampedCollector[CRow]]
 
@@ -178,6 +203,9 @@ abstract class RowTimeUnboundedOver(
         ctx.timerService.registerEventTimeTimer(curWatermark + 1)
       }
     }
+
+    // update cleanup timer
+    registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
   }
 
   /**
@@ -221,11 +249,13 @@ abstract class RowTimeUnboundedOver(
 class RowTimeUnboundedRowsOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType: TypeInformation[Row],
-    inputType: TypeInformation[CRow])
+    inputType: TypeInformation[CRow],
+    queryConfig: StreamQueryConfig)
   extends RowTimeUnboundedOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType,
-    inputType) {
+    inputType,
+    queryConfig) {
 
   override def processElementsWithSameTimestamp(
     curRowList: JList[Row],
@@ -259,11 +289,13 @@ class RowTimeUnboundedRowsOver(
 class RowTimeUnboundedRangeOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType: TypeInformation[Row],
-    inputType: TypeInformation[CRow])
+    inputType: TypeInformation[CRow],
+    queryConfig: StreamQueryConfig)
   extends RowTimeUnboundedOver(
     genAggregations: GeneratedAggregationsFunction,
     intermediateType,
-    inputType) {
+    inputType,
+    queryConfig) {
 
   override def processElementsWithSameTimestamp(
     curRowList: JList[Row],


[4/5] flink git commit: [FLINK-6483] [table] Add materialization of time indicators.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala
new file mode 100644
index 0000000..7ac0874
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala
@@ -0,0 +1,351 @@
+/*
+ * 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 java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.RelTimeIndicatorConverterTest.TableFunc
+import org.apache.flink.table.expressions.{TimeIntervalUnit, WindowReference}
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.plan.logical.TumblingGroupWindow
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+/**
+  * Tests for [[RelTimeIndicatorConverter]].
+  */
+class RelTimeIndicatorConverterTest extends TableTestBase {
+
+  @Test
+  def testSimpleMaterialization(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+
+    val result = t
+      .select('rowtime.floor(TimeIntervalUnit.DAY) as 'rowtime, 'long)
+      .filter('long > 0)
+      .select('rowtime)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select", "FLOOR(TIME_MATERIALIZATION(rowtime)", "FLAG(DAY)) AS rowtime"),
+      term("where", ">(long, 0)")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testSelectAll(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+
+    val result = t.select('*)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime", "long", "int",
+        "TIME_MATERIALIZATION(proctime) AS proctime")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testFilteringOnRowtime(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int)
+
+    val result = t
+      .filter('rowtime > "1990-12-02 12:11:11".toTimestamp)
+      .select('rowtime)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime"),
+      term("where", ">(TIME_MATERIALIZATION(rowtime), 1990-12-02 12:11:11)")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testGroupingOnRowtime(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+
+    val result = t
+      .groupBy('rowtime)
+      .select('long.count)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "long", "TIME_MATERIALIZATION(rowtime) AS rowtime")
+        ),
+        term("groupBy", "rowtime"),
+        term("select", "rowtime", "COUNT(long) AS TMP_0")
+      ),
+      term("select", "TMP_0")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testGroupingOnProctimeSql(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int)]("MyTable" , 'long, 'int, 'proctime.proctime)
+
+    val result = util.tEnv.sql("SELECT COUNT(long) FROM MyTable GROUP BY proctime")
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "TIME_MATERIALIZATION(proctime) AS proctime", "long")
+        ),
+        term("groupBy", "proctime"),
+        term("select", "proctime", "COUNT(long) AS EXPR$0")
+      ),
+      term("select", "EXPR$0")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testAggregationOnRowtime(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int)
+
+    val result = t
+      .groupBy('long)
+      .select('rowtime.min)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime", "long")
+        ),
+        term("groupBy", "long"),
+        term("select", "long", "MIN(rowtime) AS TMP_0")
+      ),
+      term("select", "TMP_0")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testAggregationOnProctimeSql(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Int)]("MyTable" , 'long, 'int, 'proctime.proctime)
+
+    val result = util.tEnv.sql("SELECT MIN(proctime) FROM MyTable GROUP BY long")
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "long", "TIME_MATERIALIZATION(proctime) AS proctime")
+        ),
+        term("groupBy", "long"),
+        term("select", "long", "MIN(proctime) AS EXPR$0")
+      ),
+      term("select", "EXPR$0")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testTableFunction(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+    val func = new TableFunc
+
+    val result = t.join(func('rowtime, 'proctime) as 's).select('rowtime, 'proctime, 's)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation",
+          s"${func.functionIdentifier}(TIME_MATERIALIZATION($$0), TIME_MATERIALIZATION($$3))"),
+        term("function", func),
+        term("rowType", "RecordType(TIMESTAMP(3) rowtime, BIGINT long, INTEGER int, " +
+          "TIMESTAMP(3) proctime, VARCHAR(2147483647) s)"),
+        term("joinType", "INNER")
+      ),
+      term("select",
+        "TIME_MATERIALIZATION(rowtime) AS rowtime",
+        "TIME_MATERIALIZATION(proctime) AS proctime",
+        "s")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testWindow(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int)
+
+    val result = t
+      .window(Tumble over 100.millis on 'rowtime as 'w)
+      .groupBy('w, 'long)
+      .select('w.end as 'rowtime, 'long, 'int.sum)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupWindowAggregate",
+        streamTableNode(0),
+        term("groupBy", "long"),
+        term(
+          "window",
+          TumblingGroupWindow(
+            WindowReference("w"),
+            'rowtime,
+            100.millis)),
+        term("select", "long", "SUM(int) AS TMP_1", "end(WindowReference(w)) AS TMP_0")
+      ),
+      term("select", "TMP_0 AS rowtime", "long", "TMP_1")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testWindowSql(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Long, Int)]("MyTable", 'rowtime.rowtime, 'long, 'int)
+
+    val result = util.tEnv.sql(
+      "SELECT TUMBLE_END(rowtime, INTERVAL '0.1' SECOND) AS `rowtime`, `long`, " +
+        "SUM(`int`) FROM MyTable " +
+        "GROUP BY `long`, TUMBLE(rowtime, INTERVAL '0.1' SECOND)")
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupWindowAggregate",
+        streamTableNode(0),
+        term("groupBy", "long"),
+        term(
+          "window",
+          TumblingGroupWindow(
+            'w$,
+            'rowtime,
+            100.millis)),
+        term("select", "long", "SUM(int) AS EXPR$2", "start('w$) AS w$start", "end('w$) AS w$end")
+      ),
+      term("select", "w$end", "long", "EXPR$2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testWindowWithAggregationOnRowtimeSql(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Long, Long, Int)]("MyTable", 'rowtime.rowtime, 'long, 'int)
+
+    val result = util.tEnv.sql("SELECT MIN(rowtime), long FROM MyTable " +
+      "GROUP BY long, TUMBLE(rowtime, INTERVAL '0.1' SECOND)")
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamGroupWindowAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "long", "1970-01-01 00:00:00 AS $f1",
+            "TIME_MATERIALIZATION(rowtime) AS $f2")
+        ),
+        term("groupBy", "long"),
+        term(
+          "window",
+          TumblingGroupWindow(
+            'w$,
+            'rowtime,
+            100.millis)),
+        term("select", "long", "MIN($f2) AS EXPR$0")
+      ),
+      term("select", "EXPR$0", "long")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testUnion(): Unit = {
+    val util = streamTestUtil()
+    val t = util.addTable[(Long, Long, Int)]("MyTable", 'rowtime.rowtime, 'long, 'int)
+
+    val result = t.unionAll(t).select('rowtime)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      binaryNode(
+        "DataStreamUnion",
+        streamTableNode(0),
+        streamTableNode(0),
+        term("union all", "rowtime", "long", "int")
+      ),
+      term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+}
+
+object RelTimeIndicatorConverterTest {
+
+  class TableFunc extends TableFunction[String] {
+    val t = new Timestamp(0L)
+    def eval(time1: Long, time2: Timestamp): Unit = {
+      collect(time1.toString + time2.after(t))
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala
new file mode 100644
index 0000000..7d7088e
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala
@@ -0,0 +1,237 @@
+/*
+ * 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.datastream
+
+import java.math.BigDecimal
+
+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.scala._
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase
+import org.apache.flink.table.api.{TableEnvironment, TableException, Types}
+import org.apache.flink.table.calcite.RelTimeIndicatorConverterTest.TableFunc
+import org.apache.flink.table.expressions.TimeIntervalUnit
+import org.apache.flink.table.runtime.datastream.TimeAttributesITCase.TimestampWithEqualWatermark
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+/**
+  * Tests for access and materialization of time attributes.
+  */
+class TimeAttributesITCase extends StreamingMultipleProgramsTestBase {
+
+  val data = List(
+    (1L, 1, 1d, 1f, new BigDecimal("1"), "Hi"),
+    (2L, 2, 2d, 2f, new BigDecimal("2"), "Hallo"),
+    (3L, 2, 2d, 2f, new BigDecimal("2"), "Hello"),
+    (4L, 5, 5d, 5f, new BigDecimal("5"), "Hello"),
+    (7L, 3, 3d, 3f, new BigDecimal("3"), "Hello"),
+    (8L, 3, 3d, 3f, new BigDecimal("3"), "Hello world"),
+    (16L, 4, 4d, 4f, new BigDecimal("4"), "Hello world"))
+
+  @Test(expected = classOf[TableException])
+  def testInvalidTimeCharacteristic(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val stream = env
+      .fromCollection(data)
+      .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+    stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+  }
+
+  @Test
+  def testCalcMaterialization(): 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 TimestampWithEqualWatermark())
+    val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+
+    val t = table.select('rowtime.cast(Types.STRING))
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "1970-01-01 00:00:00.001",
+      "1970-01-01 00:00:00.002",
+      "1970-01-01 00:00:00.003",
+      "1970-01-01 00:00:00.004",
+      "1970-01-01 00:00:00.007",
+      "1970-01-01 00:00:00.008",
+      "1970-01-01 00:00:00.016")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testCalcMaterialization2(): 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 TimestampWithEqualWatermark())
+    val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+
+    val t = table
+      .filter('rowtime.cast(Types.LONG) > 4)
+      .select('rowtime, 'rowtime.floor(TimeIntervalUnit.DAY), 'rowtime.ceil(TimeIntervalUnit.DAY))
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "1970-01-01 00:00:00.007,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0",
+      "1970-01-01 00:00:00.008,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0",
+      "1970-01-01 00:00:00.016,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testTableFunction(): 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 TimestampWithEqualWatermark())
+    val table = stream.toTable(
+      tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'proctime.proctime)
+    val func = new TableFunc
+
+    val t = table.join(func('rowtime, 'proctime) as 's).select('rowtime, 's)
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "1970-01-01 00:00:00.001,1true",
+      "1970-01-01 00:00:00.002,2true",
+      "1970-01-01 00:00:00.003,3true",
+      "1970-01-01 00:00:00.004,4true",
+      "1970-01-01 00:00:00.007,7true",
+      "1970-01-01 00:00:00.008,8true",
+      "1970-01-01 00:00:00.016,16true")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testUnion(): 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 TimestampWithEqualWatermark())
+    val table = stream.toTable(
+      tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+
+    val t = table.unionAll(table).select('rowtime)
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "1970-01-01 00:00:00.001",
+      "1970-01-01 00:00:00.001",
+      "1970-01-01 00:00:00.002",
+      "1970-01-01 00:00:00.002",
+      "1970-01-01 00:00:00.003",
+      "1970-01-01 00:00:00.003",
+      "1970-01-01 00:00:00.004",
+      "1970-01-01 00:00:00.004",
+      "1970-01-01 00:00:00.007",
+      "1970-01-01 00:00:00.007",
+      "1970-01-01 00:00:00.008",
+      "1970-01-01 00:00:00.008",
+      "1970-01-01 00:00:00.016",
+      "1970-01-01 00:00:00.016")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testWindowWithAggregationOnRowtimeSql(): 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 TimestampWithEqualWatermark())
+    val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+    tEnv.registerTable("MyTable", table)
+
+    val t = tEnv.sql("SELECT COUNT(`rowtime`) FROM MyTable " +
+      "GROUP BY TUMBLE(rowtime, INTERVAL '0.003' SECOND)")
+
+    val results = t.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "1",
+      "2",
+      "2",
+      "2"
+    )
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+}
+
+object TimeAttributesITCase {
+  class TimestampWithEqualWatermark
+  extends AssignerWithPunctuatedWatermarks[(Long, Int, Double, Float, BigDecimal, String)] {
+
+    override def checkAndGetNextWatermark(
+        lastElement: (Long, Int, Double, Float, BigDecimal, String),
+        extractedTimestamp: Long)
+      : Watermark = {
+      new Watermark(extractedTimestamp)
+    }
+
+    override def extractTimestamp(
+        element: (Long, Int, Double, Float, BigDecimal, String),
+        previousElementTimestamp: Long): Long = {
+      element._1
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/609d5a32/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 0e6d461..65014cd 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
@@ -24,9 +24,11 @@ import org.apache.flink.api.java.{DataSet => JDataSet}
 import org.apache.flink.table.api.{Table, TableEnvironment}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
+import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.table.expressions.Expression
 import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
 import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream}
+import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JStreamExecutionEnvironment}
 import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
 import org.apache.flink.table.api.scala.batch.utils.LogicalPlanFormatUtils
 import org.junit.Assert.assertEquals
@@ -174,7 +176,10 @@ case class BatchTableTestUtil() extends TableTestUtil {
 
 case class StreamTableTestUtil() extends TableTestUtil {
 
+  val javaEnv = mock(classOf[JStreamExecutionEnvironment])
+  when(javaEnv.getStreamTimeCharacteristic).thenReturn(TimeCharacteristic.EventTime)
   val env = mock(classOf[StreamExecutionEnvironment])
+  when(env.getWrappedStreamExecutionEnvironment).thenReturn(javaEnv)
   val tEnv = TableEnvironment.getTableEnvironment(env)
 
   def addTable[T: TypeInformation](


[3/5] flink git commit: [FLINK-6491] [table] Add QueryConfig and state clean up for non-windowed aggregates.

Posted by fh...@apache.org.
[FLINK-6491] [table] Add QueryConfig and state clean up 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/003f81a7
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/003f81a7
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/003f81a7

Branch: refs/heads/release-1.3
Commit: 003f81a73fe38edcd17f10f8e6afb50ba23e3c28
Parents: f3ce088
Author: Fabian Hueske <fh...@apache.org>
Authored: Mon May 8 18:41:37 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri May 12 08:33:44 2017 +0200

----------------------------------------------------------------------
 .../flink/table/api/BatchTableEnvironment.scala |  17 ++-
 .../apache/flink/table/api/QueryConfig.scala    | 102 ++++++++++++++++
 .../table/api/StreamTableEnvironment.scala      |  51 ++++++--
 .../flink/table/api/TableEnvironment.scala      |   2 +-
 .../table/api/java/StreamTableEnvironment.scala | 115 +++++++++++++++++--
 .../api/scala/StreamTableEnvironment.scala      |  46 +++++++-
 .../table/api/scala/TableConversions.scala      |  40 ++++++-
 .../org/apache/flink/table/api/table.scala      |  26 ++++-
 .../plan/nodes/datastream/DataStreamCalc.scala  |   8 +-
 .../nodes/datastream/DataStreamCorrelate.scala  |   8 +-
 .../datastream/DataStreamGroupAggregate.scala   |  20 +++-
 .../DataStreamGroupWindowAggregate.scala        |   8 +-
 .../datastream/DataStreamOverAggregate.scala    |   9 +-
 .../plan/nodes/datastream/DataStreamRel.scala   |   7 +-
 .../plan/nodes/datastream/DataStreamScan.scala  |   7 +-
 .../plan/nodes/datastream/DataStreamUnion.scala |  10 +-
 .../nodes/datastream/DataStreamValues.scala     |   6 +-
 .../datastream/StreamTableSourceScan.scala      |   7 +-
 .../table/runtime/aggregate/AggregateUtil.scala |   6 +-
 .../aggregate/GroupAggProcessFunction.scala     |  54 ++++++++-
 .../table/utils/MockTableEnvironment.scala      |   9 +-
 21 files changed, 494 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 2a3cedf..f33c187 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
@@ -32,7 +32,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, RowtimeAttribute, TimeAttribute}
+import org.apache.flink.table.expressions.{Expression, 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
@@ -113,9 +113,20 @@ abstract class BatchTableEnvironment(
     *
     * @param table The [[Table]] to write.
     * @param sink The [[TableSink]] to write the [[Table]] to.
+    * @param qConfig The configuration for the query to generate.
     * @tparam T The expected type of the [[DataSet]] which represents the [[Table]].
     */
-  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
+  override private[flink] def writeToSink[T](
+      table: Table,
+      sink: TableSink[T],
+      qConfig: QueryConfig): Unit = {
+
+    // We do not pass the configuration on, because there is nothing to configure for batch queries.
+    val bQConfig = qConfig match {
+      case batchConfig: BatchQueryConfig => batchConfig
+      case _ =>
+        throw new TableException("BatchQueryConfig required to configure batch query.")
+    }
 
     sink match {
       case batchSink: BatchTableSink[T] =>
@@ -125,7 +136,7 @@ abstract class BatchTableEnvironment(
         // Give the DataSet to the TableSink to emit it.
         batchSink.emitDataSet(result)
       case _ =>
-        throw new TableException("BatchTableSink required to emit batch Table")
+        throw new TableException("BatchTableSink required to emit batch Table.")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
new file mode 100644
index 0000000..8e8b5ac
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import _root_.java.io.Serializable
+import org.apache.flink.api.common.time.Time
+
+class QueryConfig private[table] extends Serializable {}
+
+/**
+  * The [[BatchQueryConfig]] holds parameters to configure the behavior of batch queries.
+  */
+class BatchQueryConfig private[table] extends QueryConfig
+
+/**
+  * The [[StreamQueryConfig]] holds parameters to configure the behavior of streaming queries.
+  *
+  * An empty [[StreamQueryConfig]] can be generated using the [[StreamTableEnvironment.qConf]]
+  * method.
+  */
+class StreamQueryConfig private[table] extends QueryConfig {
+
+  /**
+    * The minimum time until state which was not updated will be retained.
+    * State might be cleared and removed if it was not updated for the defined period of time.
+    */
+  private var minIdleStateRetentionTime: Long = Long.MinValue
+
+  /**
+    * The maximum time until state which was not updated will be retained.
+    * State will be cleared and removed if it was not updated for the defined period of time.
+    */
+  private var maxIdleStateRetentionTime: Long = Long.MinValue
+
+  /**
+    * Specifies the time interval for how long idle state, i.e., state which was not updated, will
+    * be retained. When state was not updated for the specified interval of time, it will be cleared
+    * and removed.
+    *
+    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
+    * was the first data. This can result in previous results being overwritten.
+    *
+    * Note: [[setIdleStateRetentionTime(minTime: Time, maxTime: Time)]] allows to set a minimum and
+    * maximum time for state to be retained. This method is more efficient, because the system has
+    * to do less bookkeeping to identify the time at which state must be cleared.
+    *
+    * @param time The time interval for how long idle state is retained. Set to 0 (zero) to never
+    *             clean-up the state.
+    */
+  def setIdleStateRetentionTime(time: Time): StreamQueryConfig = {
+    setIdleStateRetentionTime(time, time)
+  }
+
+  /**
+    * Specifies a minimum and a maximum time interval for how long idle state, i.e., state which
+    * was not updated, will be retained.
+    * State will never be cleared until it was idle for less than the minimum time and will never
+    * be kept if it was idle for more than the maximum time.
+    *
+    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
+    * was the first data. This can result in previous results being overwritten.
+    *
+    * Set to 0 (zero) to never clean-up the state.
+    *
+    * @param minTime The minimum time interval for which idle state is retained. Set to 0 (zero) to
+    *                never clean-up the state.
+    * @param maxTime The maximum time interval for which idle state is retained. May not be smaller
+    *                than than minTime. Set to 0 (zero) to never clean-up the state.
+    */
+  def setIdleStateRetentionTime(minTime: Time, maxTime: Time): StreamQueryConfig = {
+    if (maxTime.toMilliseconds < minTime.toMilliseconds) {
+      throw new IllegalArgumentException("maxTime may not be smaller than minTime.")
+    }
+    minIdleStateRetentionTime = minTime.toMilliseconds
+    maxIdleStateRetentionTime = maxTime.toMilliseconds
+    this
+  }
+
+  def getMinIdleStateRetentionTime: Long = {
+    minIdleStateRetentionTime
+  }
+
+  def getMaxIdleStateRetentionTime: Long = {
+    maxIdleStateRetentionTime
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 aef2b1b..c594d4c 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
@@ -81,6 +81,8 @@ abstract class StreamTableEnvironment(
   // the naming pattern for internally registered tables.
   private val internalNamePattern = "^_DataStreamTable_[0-9]+$".r
 
+  def qConf: StreamQueryConfig = new StreamQueryConfig
+
   /**
     * Checks if the chosen table name is valid.
     *
@@ -126,9 +128,20 @@ abstract class StreamTableEnvironment(
     *
     * @param table The [[Table]] to write.
     * @param sink The [[TableSink]] to write the [[Table]] to.
+    * @param qConfig The configuration for the query to generate.
     * @tparam T The expected type of the [[DataStream]] which represents the [[Table]].
     */
-  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
+  override private[flink] def writeToSink[T](
+      table: Table,
+      sink: TableSink[T],
+      qConfig: QueryConfig): Unit = {
+
+    // Check query configuration
+    val sQConf = qConfig match {
+      case streamConfig: StreamQueryConfig => streamConfig
+      case _ =>
+        throw new TableException("StreamQueryConfig required to configure stream query.")
+    }
 
     sink match {
 
@@ -137,7 +150,7 @@ abstract class StreamTableEnvironment(
         val outputType = sink.getOutputType
         // translate the Table into a DataStream and provide the type that the TableSink expects.
         val result: DataStream[T] =
-          translate(table, updatesAsRetraction = true, withChangeFlag = true)(outputType)
+          translate(table, sQConf, 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]]])
@@ -160,7 +173,11 @@ abstract class StreamTableEnvironment(
         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)
+          translate(
+            optimizedPlan,
+            table.getRelNode.getRowType,
+            sQConf,
+            withChangeFlag = true)(outputType)
         // Give the DataStream to the TableSink to emit it.
         upsertSink.asInstanceOf[UpsertStreamTableSink[Any]]
           .emitDataStream(result.asInstanceOf[DataStream[JTuple2[JBool, Any]]])
@@ -176,7 +193,11 @@ abstract class StreamTableEnvironment(
         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 = false)(outputType)
+          translate(
+            optimizedPlan,
+            table.getRelNode.getRowType,
+            sQConf,
+            withChangeFlag = false)(outputType)
         // Give the DataStream to the TableSink to emit it.
         appendSink.asInstanceOf[AppendStreamTableSink[T]].emitDataStream(result)
 
@@ -545,17 +566,21 @@ 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 qConfig The configuration for the query to generate.
     * @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, updatesAsRetraction: Boolean, withChangeFlag: Boolean)
-      (implicit tpe: TypeInformation[A]): DataStream[A] = {
+  protected def translate[A](
+      table: Table,
+      qConfig: StreamQueryConfig,
+      updatesAsRetraction: Boolean,
+      withChangeFlag: Boolean)(implicit tpe: TypeInformation[A]): DataStream[A] = {
     val relNode = table.getRelNode
     val dataStreamPlan = optimize(relNode, updatesAsRetraction)
-    translate(dataStreamPlan, relNode.getRowType, withChangeFlag)
+    translate(dataStreamPlan, relNode.getRowType, qConfig, withChangeFlag)
   }
 
   /**
@@ -564,6 +589,7 @@ 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 qConfig     The configuration for the query to generate.
     * @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]].
@@ -572,6 +598,7 @@ abstract class StreamTableEnvironment(
   protected def translate[A](
       logicalPlan: RelNode,
       logicalType: RelDataType,
+      qConfig: StreamQueryConfig,
       withChangeFlag: Boolean)
       (implicit tpe: TypeInformation[A]): DataStream[A] = {
 
@@ -583,7 +610,7 @@ abstract class StreamTableEnvironment(
     }
 
     // get CRow plan
-    val plan: DataStream[CRow] = translateToCRow(logicalPlan)
+    val plan: DataStream[CRow] = translateToCRow(logicalPlan, qConfig)
 
     // convert CRow to output type
     val conversion = if (withChangeFlag) {
@@ -615,14 +642,16 @@ abstract class StreamTableEnvironment(
     * Translates a logical [[RelNode]] plan into a [[DataStream]] of type [[CRow]].
     *
     * @param logicalPlan The logical plan to translate.
+    * @param qConfig The configuration for the query to generate.
     * @return The [[DataStream]] of type [[CRow]].
     */
   protected def translateToCRow(
-    logicalPlan: RelNode): DataStream[CRow] = {
+    logicalPlan: RelNode,
+    qConfig: StreamQueryConfig): DataStream[CRow] = {
 
     logicalPlan match {
       case node: DataStreamRel =>
-        node.translateToPlan(this)
+        node.translateToPlan(this, qConfig)
       case _ =>
         throw TableException("Cannot generate DataStream due to an invalid logical plan. " +
           "This is a bug and should not happen. Please file an issue.")
@@ -638,7 +667,7 @@ abstract class StreamTableEnvironment(
   def explain(table: Table): String = {
     val ast = table.getRelNode
     val optimizedPlan = optimize(ast, updatesAsRetraction = false)
-    val dataStream = translateToCRow(optimizedPlan)
+    val dataStream = translateToCRow(optimizedPlan, qConf)
 
     val env = dataStream.getExecutionEnvironment
     val jsonSqlPlan = env.getExecutionPlan

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 bf4a8e0..9f50f0c 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
@@ -510,7 +510,7 @@ abstract class TableEnvironment(val config: TableConfig) {
     * @param sink The [[TableSink]] to write the [[Table]] to.
     * @tparam T The data type that the [[TableSink]] expects.
     */
-  private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit
+  private[flink] def writeToSink[T](table: Table, sink: TableSink[T], conf: QueryConfig): Unit
 
   /**
     * Registers a Calcite [[AbstractTable]] in the TableEnvironment's catalog.

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 a70bcca..c3b5951 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
@@ -150,9 +150,50 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = {
+    toDataStream(table, clazz, qConf)
+  }
+
+  /**
+    * 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.
+    * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match.
+    *
+    * @param table The [[Table]] to convert.
+    * @param typeInfo The [[TypeInformation]] that specifies the type of the [[DataStream]].
+    * @tparam T The type of the resulting [[DataStream]].
+    * @return The converted [[DataStream]].
+    */
+  def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = {
+    toDataStream(table, typeInfo, qConf)
+  }
+
+  /**
+    * 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.
+    * - 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 type of the resulting [[DataStream]].
+    * @param qConfig The configuration of the query to generate.
+    * @tparam T The type of the resulting [[DataStream]].
+    * @return The converted [[DataStream]].
+    */
+  def toDataStream[T](table: Table, clazz: Class[T], qConfig: StreamQueryConfig): DataStream[T] = {
     val typeInfo = TypeExtractor.createTypeInfo(clazz)
     TableEnvironment.validateType(typeInfo)
-    translate[T](table, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
+    translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
   }
 
   /**
@@ -168,12 +209,64 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param typeInfo The [[TypeInformation]] that specifies the type of the [[DataStream]].
+    * @param qConfig The configuration of the query to generate.
     * @tparam T The type of the resulting [[DataStream]].
     * @return The converted [[DataStream]].
     */
-  def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = {
+  def toDataStream[T](
+      table: Table,
+      typeInfo: TypeInformation[T],
+      qConfig: StreamQueryConfig): DataStream[T] = {
     TableEnvironment.validateType(typeInfo)
-    translate[T](table, updatesAsRetraction = false, withChangeFlag = false)(typeInfo)
+    translate[T](table, qConfig, 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]] = {
+
+    toRetractStream(table, clazz, qConf)
+  }
+
+  /**
+    * 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]] = {
+
+    toRetractStream(table, typeInfo, qConf)
   }
 
   /**
@@ -190,17 +283,21 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param clazz The class of the requested record type.
+    * @param qConfig The configuration of the query to generate.
     * @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]] = {
+  def toRetractStream[T](
+      table: Table,
+      clazz: Class[T],
+      qConfig: StreamQueryConfig): 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,
+      qConfig,
       updatesAsRetraction = true,
       withChangeFlag = true)(resultType)
   }
@@ -219,11 +316,14 @@ class StreamTableEnvironment(
     *
     * @param table The [[Table]] to convert.
     * @param typeInfo The [[TypeInformation]] of the requested record type.
+    * @param qConfig The configuration of the query to generate.
     * @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]] = {
+  def toRetractStream[T](
+      table: Table,
+      typeInfo: TypeInformation[T],
+      qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = {
 
     TableEnvironment.validateType(typeInfo)
     val resultTypeInfo = new TupleTypeInfo[JTuple2[JBool, T]](
@@ -232,6 +332,7 @@ class StreamTableEnvironment(
     )
     translate[JTuple2[JBool, T]](
       table,
+      qConfig,
       updatesAsRetraction = true,
       withChangeFlag = true)(resultTypeInfo)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 e5ad6c2..56f7d55 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
@@ -19,7 +19,7 @@ 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.{Table, TableConfig, TableEnvironment}
+import org.apache.flink.table.api.{StreamQueryConfig, 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}
@@ -143,8 +143,29 @@ class StreamTableEnvironment(
     * @return The converted [[DataStream]].
     */
   def toDataStream[T: TypeInformation](table: Table): DataStream[T] = {
+    toDataStream(table, qConf)
+  }
+
+  /**
+    * 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 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.
+    * @param qConfig The configuration of the query to generate.
+    * @tparam T The type of the resulting [[DataStream]].
+    * @return The converted [[DataStream]].
+    */
+  def toDataStream[T: TypeInformation](table: Table, qConfig: StreamQueryConfig): DataStream[T] = {
     val returnType = createTypeInformation[T]
-    asScalaStream(translate(table, updatesAsRetraction = false, withChangeFlag = false)(returnType))
+    asScalaStream(
+      translate(table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(returnType))
   }
 
 /**
@@ -159,8 +180,27 @@ class StreamTableEnvironment(
   * @return The converted [[DataStream]].
   */
   def toRetractStream[T: TypeInformation](table: Table): DataStream[(Boolean, T)] = {
+    toRetractStream(table, qConf)
+  }
+
+  /**
+    * 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.
+    * @param qConfig The configuration of the query to generate.
+    * @tparam T The type of the requested data type.
+    * @return The converted [[DataStream]].
+    */
+  def toRetractStream[T: TypeInformation](
+      table: Table,
+      qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = {
     val returnType = createTypeInformation[(Boolean, T)]
-    asScalaStream(translate(table, updatesAsRetraction = true, withChangeFlag = true)(returnType))
+    asScalaStream(
+      translate(table, qConfig, updatesAsRetraction = true, withChangeFlag = true)(returnType))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 5efff62..966b42f 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
@@ -21,7 +21,7 @@ package org.apache.flink.table.api.scala
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.streaming.api.scala.DataStream
-import org.apache.flink.table.api.{Table, TableException}
+import org.apache.flink.table.api.{StreamQueryConfig, Table, TableException}
 import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv}
 import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv}
 
@@ -57,6 +57,21 @@ class TableConversions(table: Table) {
     }
   }
 
+  /** Converts the [[Table]] to a [[DataStream]] of the specified type.
+    *
+    * @param qConfig The configuration for the generated query.
+    */
+  def toDataStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[T] = {
+    table.tableEnv match {
+      case tEnv: ScalaStreamTableEnv =>
+        tEnv.toDataStream(table, qConfig)
+      case _ =>
+        throw new TableException(
+          "Only tables that originate from Scala DataStreams " +
+            "can be converted to Scala DataStreams.")
+    }
+  }
+
   /** 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]].
@@ -76,5 +91,28 @@ 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.
+    *
+    * @param qConfig The configuration for the generated query.
+    *
+    */
+  def toRetractStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = {
+
+    table.tableEnv match {
+      case tEnv: ScalaStreamTableEnv =>
+        tEnv.toRetractStream(table, qConfig)
+      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/003f81a7/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 310a75f..5a2eb1c 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
@@ -763,6 +763,30 @@ class Table(
     * @tparam T The data type that the [[TableSink]] expects.
     */
   def writeToSink[T](sink: TableSink[T]): Unit = {
+
+    def qConfig = this.tableEnv match {
+      case s: StreamTableEnvironment => s.qConf
+      case b: BatchTableEnvironment => new BatchQueryConfig
+      case _ => null
+    }
+
+    writeToSink(sink, qConfig)
+  }
+
+  /**
+    * Writes the [[Table]] to a [[TableSink]]. A [[TableSink]] defines an external storage location.
+    *
+    * 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.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.
+    * @param conf The configuration for the query that writes to the sink.
+    * @tparam T The data type that the [[TableSink]] expects.
+    */
+  def writeToSink[T](sink: TableSink[T], conf: QueryConfig): Unit = {
     // get schema information of table
     val rowType = getRelNode.getRowType
     val fieldNames: Array[String] = rowType.getFieldNames.asScala.toArray
@@ -773,7 +797,7 @@ class Table(
     val configuredSink = sink.configure(fieldNames, fieldTypes)
 
     // emit the table to the configured table sink
-    tableEnv.writeToSink(this, configuredSink)
+    tableEnv.writeToSink(this, configuredSink, conf)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 ce0f966..0e377b5 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
@@ -25,7 +25,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rel.{RelNode, RelWriter}
 import org.apache.calcite.rex.RexProgram
 import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment}
 import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.nodes.CommonCalc
@@ -83,11 +83,13 @@ class DataStreamCalc(
     estimateRowCount(calcProgram, rowCnt)
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
-    val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
     val inputRowType = inputDataStream.getType.asInstanceOf[CRowTypeInfo].rowType
 
     val generator = new CodeGenerator(config, false, inputRowType)

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 19ad89b..cbd818a 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
@@ -23,7 +23,7 @@ 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.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.StreamTableEnvironment
+import org.apache.flink.table.api.{StreamQueryConfig, 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
@@ -82,12 +82,14 @@ class DataStreamCorrelate(
       .itemIf("condition", condition.orNull, condition.isDefined)
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 
     // we do not need to specify input type
-    val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
     val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo]
 
     val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 506c0cb..f01b24a 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
@@ -21,9 +21,10 @@ 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.common.time.Time
 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.api.{StreamQueryConfig, StreamTableEnvironment}
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.runtime.aggregate._
 import org.apache.flink.table.plan.nodes.CommonAggregate
@@ -31,6 +32,7 @@ import org.apache.flink.table.plan.schema.RowSchema
 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}
+import org.slf4j.LoggerFactory
 
 /**
   *
@@ -59,6 +61,8 @@ class DataStreamGroupAggregate(
     with CommonAggregate
     with DataStreamRel {
 
+  private val LOG = LoggerFactory.getLogger(this.getClass)
+
   override def deriveRowType() = schema.logicalType
 
   override def needsUpdatesAsRetraction = true
@@ -100,9 +104,18 @@ class DataStreamGroupAggregate(
         inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil))
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
+
+    if (qConfig.getMinIdleStateRetentionTime < 0 || qConfig.getMaxIdleStateRetentionTime < 0) {
+      LOG.warn(
+        "No state retention interval configured for a query which accumulates state. " +
+        "Please provide a query configuration with valid retention interval to prevent excessive " +
+          "state size. You may specify a retention time of 0 to not clean up the state.")
+    }
 
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
 
     val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
       new CalcitePair[AggregateCall, String](
@@ -136,6 +149,7 @@ class DataStreamGroupAggregate(
       inputSchema.logicalType,
       inputSchema.physicalFieldTypeInfo,
       groupings,
+      qConfig,
       DataStreamRetractionRules.isAccRetract(this),
       DataStreamRetractionRules.isAccRetract(getInput))
 

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 ef207b0..d2aaad0 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, TableException}
+import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException}
 import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.expressions.ExpressionUtils._
@@ -107,9 +107,11 @@ class DataStreamGroupWindowAggregate(
           namedProperties))
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
 
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
 
     val physicalNamedAggregates = namedAggregates.map { namedAggregate =>
       new CalcitePair[AggregateCall, String](

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 4061242..8e97884 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
@@ -26,7 +26,7 @@ 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.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
+import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.nodes.OverAggregate
 import org.apache.flink.table.plan.schema.RowSchema
@@ -88,7 +88,10 @@ class DataStreamOverAggregate(
           namedAggregates))
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
+
     if (logicWindow.groups.size > 1) {
       throw new TableException(
         "Unsupported use of OVER windows. All aggregates must be computed on the same window.")
@@ -109,7 +112,7 @@ class DataStreamOverAggregate(
         "Unsupported use of OVER windows. The window can only be ordered in ASCENDING mode.")
     }
 
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
 
     val consumeRetraction = DataStreamRetractionRules.isAccRetract(input)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 9754de4..6f6edf7 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
@@ -19,7 +19,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.api.{StreamQueryConfig, StreamTableEnvironment}
 import org.apache.flink.table.plan.nodes.FlinkRelNode
 import org.apache.flink.table.runtime.types.CRow
 
@@ -29,9 +29,12 @@ trait DataStreamRel extends FlinkRelNode {
     * Translates the FlinkRelNode into a Flink operator.
     *
     * @param tableEnv The [[StreamTableEnvironment]] of the translated Table.
+    * @param qConfig The configuration for the query to generate.
     * @return DataStream of type [[CRow]]
     */
-  def translateToPlan(tableEnv: StreamTableEnvironment) : DataStream[CRow]
+  def translateToPlan(
+    tableEnv: StreamTableEnvironment,
+    qConfig: StreamQueryConfig): 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/003f81a7/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 c613646..e64bf0f 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
@@ -23,7 +23,7 @@ import org.apache.calcite.rel.RelNode
 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.api.{StreamQueryConfig, StreamTableEnvironment}
 import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.plan.schema.DataStreamTable
 import org.apache.flink.table.runtime.types.CRow
@@ -54,7 +54,10 @@ class DataStreamScan(
     )
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): 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/003f81a7/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 654c259..6cc7396 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
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.nodes.datastream
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 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.api.{StreamQueryConfig, StreamTableEnvironment}
 import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.types.CRow
 
@@ -58,10 +58,12 @@ class DataStreamUnion(
     s"Union All(union: (${schema.logicalFieldNames.mkString(", ")}))"
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
 
-    val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-    val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
+    val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig)
     leftDataSet.union(rightDataSet)
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 32c9aaf..ba6b025 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
@@ -24,7 +24,7 @@ import org.apache.calcite.rel.RelNode
 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.api.{StreamQueryConfig, StreamTableEnvironment}
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.plan.schema.RowSchema
 import org.apache.flink.table.runtime.io.CRowValuesInputFormat
@@ -56,7 +56,9 @@ class DataStreamValues(
     )
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): DataStream[CRow] = {
 
     val config = tableEnv.getConfig
 

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 b2d7019..225f23f 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,7 +22,7 @@ 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, TableEnvironment}
+import org.apache.flink.table.api.{StreamQueryConfig, 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
@@ -98,7 +98,10 @@ class StreamTableSourceScan(
     )
   }
 
-  override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = {
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      qConfig: StreamQueryConfig): 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/003f81a7/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 768c9cb..27392c7 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
@@ -33,7 +33,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo
 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.api.{StreamQueryConfig, TableException}
 import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenerator
@@ -155,6 +155,7 @@ object AggregateUtil {
       inputRowType: RelDataType,
       inputFieldTypes: Seq[TypeInformation[_]],
       groupings: Array[Int],
+      qConfig: StreamQueryConfig,
       generateRetraction: Boolean,
       consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = {
 
@@ -190,7 +191,8 @@ object AggregateUtil {
     new GroupAggProcessFunction(
       genFunction,
       aggregationStateType,
-      generateRetraction)
+      generateRetraction,
+      qConfig)
 
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 6ee37e6..84fee87 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,9 +26,9 @@ 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.api.{StreamQueryConfig, Types}
 import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
-import org.slf4j.LoggerFactory
+import org.slf4j.{Logger, LoggerFactory}
 import org.apache.flink.table.runtime.types.CRow
 
 /**
@@ -40,13 +40,20 @@ import org.apache.flink.table.runtime.types.CRow
 class GroupAggProcessFunction(
     private val genAggregations: GeneratedAggregationsFunction,
     private val aggregationStateType: RowTypeInfo,
-    private val generateRetraction: Boolean)
+    private val generateRetraction: Boolean,
+    private val qConfig: StreamQueryConfig)
   extends ProcessFunction[CRow, CRow]
     with Compiler[GeneratedAggregations] {
 
-  val LOG = LoggerFactory.getLogger(this.getClass)
+  val LOG: Logger = LoggerFactory.getLogger(this.getClass)
   private var function: GeneratedAggregations = _
 
+  private val minRetentionTime = qConfig.getMinIdleStateRetentionTime
+  private val maxRetentionTime = qConfig.getMaxIdleStateRetentionTime
+  private val stateCleaningEnabled = minRetentionTime > 1 && maxRetentionTime > 1
+  // interval in which clean-up timers are registered
+  private val cleanupTimerInterval = maxRetentionTime - minRetentionTime
+
   private var newRow: CRow = _
   private var prevRow: CRow = _
   private var firstRow: Boolean = _
@@ -54,6 +61,8 @@ class GroupAggProcessFunction(
   private var state: ValueState[Row] = _
   // counts the number of added and retracted input records
   private var cntState: ValueState[JLong] = _
+  // holds the latest registered cleanup timer
+  private var cleanupTimeState: ValueState[JLong] = _
 
   override def open(config: Configuration) {
     LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " +
@@ -74,6 +83,12 @@ class GroupAggProcessFunction(
     val inputCntDescriptor: ValueStateDescriptor[JLong] =
       new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG)
     cntState = getRuntimeContext.getState(inputCntDescriptor)
+
+    if (stateCleaningEnabled) {
+      val inputCntDescriptor: ValueStateDescriptor[JLong] =
+        new ValueStateDescriptor[JLong]("GroupAggregateCleanupTime", Types.LONG)
+      cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor)
+    }
   }
 
   override def processElement(
@@ -81,6 +96,23 @@ class GroupAggProcessFunction(
       ctx: ProcessFunction[CRow, CRow]#Context,
       out: Collector[CRow]): Unit = {
 
+    if (stateCleaningEnabled) {
+
+      val currentTime = ctx.timerService().currentProcessingTime()
+      val earliestCleanup = currentTime + minRetentionTime
+
+      // last registered timer
+      val lastCleanupTime = cleanupTimeState.value()
+
+      if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) {
+        // we need to register a new timer
+        val cleanupTime = earliestCleanup + cleanupTimerInterval
+        // register timer and remember clean-up time
+        ctx.timerService().registerProcessingTimeTimer(cleanupTime)
+        cleanupTimeState.update(cleanupTime)
+      }
+    }
+
     val input = inputC.row
 
     // get accumulators and input counter
@@ -144,4 +176,18 @@ class GroupAggProcessFunction(
       cntState.clear()
     }
   }
+
+  override def onTimer(
+      timestamp: Long,
+      ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+      out: Collector[CRow]): Unit = {
+
+    if (timestamp == cleanupTimeState.value()) {
+      // clear all state
+      this.state.clear()
+      this.cntState.clear()
+      this.cleanupTimeState.clear()
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/003f81a7/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 8626b07..3d79e22 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,16 +18,17 @@
 
 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.api.{QueryConfig, Table, TableConfig, TableEnvironment}
 import org.apache.flink.table.sinks.TableSink
 import org.apache.flink.table.sources.TableSource
 
 class MockTableEnvironment extends TableEnvironment(new TableConfig) {
 
-  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = ???
+  override private[flink] def writeToSink[T](
+      table: Table,
+      sink: TableSink[T],
+      qConfig: QueryConfig): Unit = ???
 
   override protected def checkValidTableName(name: String): Unit = ???